diff --git a/README.md b/README.md index 1f77aa3767..a94d2c7d69 100644 --- a/README.md +++ b/README.md @@ -46,30 +46,23 @@ The following chart shows the time it takes to run the 22 TPC-H queries against using a single executor with 8 cores. See the [Comet Benchmarking Guide](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html) for details of the environment used for these benchmarks. -When using Comet, the overall run time is reduced from 615 seconds to 364 seconds, a 1.7x speedup, with query 1 -running 9x faster than Spark. +When using Comet, the overall run time is reduced from 640 seconds to 331 seconds, very close to a 2x speedup. -Running the same queries with DataFusion standalone (without Spark) using the same number of cores results in a 3.6x -speedup compared to Spark. +![](docs/source/_static/images/benchmark-results/0.5.0/tpch_allqueries.png) -Comet is not yet achieving full DataFusion speeds in all cases, but with future work we aim to provide a 2x-4x speedup -for a broader set of queries. +Here is a breakdown showing relative performance of Spark and Comet for each TPC-H query. -![](docs/source/_static/images/benchmark-results/0.4.0/tpch_allqueries.png) - -Here is a breakdown showing relative performance of Spark, Comet, and DataFusion for each TPC-H query. - -![](docs/source/_static/images/benchmark-results/0.4.0/tpch_queries_compare.png) +![](docs/source/_static/images/benchmark-results/0.5.0/tpch_queries_compare.png) The following charts shows how much Comet currently accelerates each query from the benchmark. ### Relative speedup -![](docs/source/_static/images/benchmark-results/0.4.0/tpch_queries_speedup_rel.png) +![](docs/source/_static/images/benchmark-results/0.5.0/tpch_queries_speedup_rel.png) ### Absolute speedup -![](docs/source/_static/images/benchmark-results/0.4.0/tpch_queries_speedup_abs.png) +![](docs/source/_static/images/benchmark-results/0.5.0/tpch_queries_speedup_abs.png) These benchmarks can be reproduced in any environment using the documentation in the [Comet Benchmarking Guide](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html). We encourage diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 1cc42f62ed..1927868a1c 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -172,28 +172,6 @@ public void close() { /** Returns a decoded {@link CometDecodedVector Comet vector}. */ public CometDecodedVector loadVector() { - // Only re-use Comet vector iff: - // 1. if we're not using dictionary encoding, since with dictionary encoding, the native - // side may fallback to plain encoding and the underlying memory address for the vector - // will change as result. - // 2. if the column type is of fixed width, in other words, string/binary are not supported - // since the native side may resize the vector and therefore change memory address. - // 3. if the last loaded vector contains null values: if values of last vector are all not - // null, Arrow C data API will skip loading the native validity buffer, therefore we - // should not re-use the vector in that case. - // 4. if the last loaded vector doesn't contain any null value, but the current vector also - // are all not null, which means we can also re-use the loaded vector. - // 5. if the new number of value is the same or smaller - if ((hadNull || currentNumNulls == 0) - && currentVector != null - && dictionary == null - && currentVector.isFixedLength() - && currentVector.numValues() >= currentNumValues) { - currentVector.setNumNulls(currentNumNulls); - currentVector.setNumValues(currentNumValues); - return currentVector; - } - LOG.debug("Reloading vector"); // Close the previous vector first to release struct memory allocated to import Arrow array & diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java index 8de2376f94..5fd348eebb 100644 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -53,13 +53,13 @@ public ConstantColumnReader( public ConstantColumnReader( DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { - super(type, descriptor, useDecimal128); + super(type, descriptor, useDecimal128, true); this.value = value; } ConstantColumnReader( DataType type, ColumnDescriptor descriptor, int batchSize, boolean useDecimal128) { - super(type, descriptor, useDecimal128); + super(type, descriptor, useDecimal128, true); this.batchSize = batchSize; initNative(); } 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 13b90e256e..2820c42f89 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -40,9 +40,14 @@ public class MetadataColumnReader extends AbstractColumnReader { private ArrowArray array = null; private ArrowSchema schema = null; - public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { + private boolean isConstant; + + public MetadataColumnReader( + DataType type, ColumnDescriptor descriptor, boolean useDecimal128, boolean isConstant) { // TODO: should we handle legacy dates & timestamps for metadata columns? super(type, descriptor, useDecimal128, false); + + this.isConstant = isConstant; } @Override @@ -62,7 +67,7 @@ public void readBatch(int total) { Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); FieldVector fieldVector = Data.importVector(allocator, array, schema, null); - vector = new CometPlainVector(fieldVector, useDecimal128); + vector = new CometPlainVector(fieldVector, useDecimal128, false, isConstant); } vector.setNumValues(total); diff --git a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java index 8448318dbd..46e6ee67f5 100644 --- a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java @@ -33,7 +33,7 @@ public class RowIndexColumnReader extends MetadataColumnReader { private long offset; public RowIndexColumnReader(StructField field, int batchSize, long[] indices) { - super(field.dataType(), TypeUtil.convertToParquet(field), false); + super(field.dataType(), TypeUtil.convertToParquet(field), false, false); this.indices = indices; setBatchSize(batchSize); } diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java index 65cc876bda..f3803d53a9 100644 --- a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -38,11 +38,18 @@ public class CometPlainVector extends CometDecodedVector { private byte booleanByteCache; private int booleanByteCacheIndex = -1; + private boolean isReused; + public CometPlainVector(ValueVector vector, boolean useDecimal128) { this(vector, useDecimal128, false); } public CometPlainVector(ValueVector vector, boolean useDecimal128, boolean isUuid) { + this(vector, useDecimal128, isUuid, false); + } + + public CometPlainVector( + ValueVector vector, boolean useDecimal128, boolean isUuid, boolean isReused) { super(vector, vector.getField(), useDecimal128, isUuid); // NullType doesn't have data buffer. if (vector instanceof NullVector) { @@ -52,6 +59,15 @@ public CometPlainVector(ValueVector vector, boolean useDecimal128, boolean isUui } isBaseFixedWidthVector = valueVector instanceof BaseFixedWidthVector; + this.isReused = isReused; + } + + public boolean isReused() { + return isReused; + } + + public void setReused(boolean isReused) { + this.isReused = isReused; } @Override diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 84089c16a1..cdfeb57710 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -210,6 +210,8 @@ object CometConf extends ShimCometConf { createExecEnabledConfig("window", defaultValue = true) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) + val COMET_EXEC_INITCAP_ENABLED: ConfigEntry[Boolean] = + createExecEnabledConfig("initCap", defaultValue = false) val COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.exec.sortMergeJoinWithJoinFilter.enabled") @@ -275,6 +277,13 @@ object CometConf extends ShimCometConf { .checkValues(Set("native", "jvm", "auto")) .createWithDefault("auto") + val COMET_SHUFFLE_FALLBACK_TO_COLUMNAR: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.fallbackToColumnar") + .doc("Whether to try falling back to columnar shuffle when native shuffle is not supported") + .internal() + .booleanConf + .createWithDefault(false) + val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( @@ -293,12 +302,29 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) - val COMET_EXEC_SHUFFLE_CODEC: ConfigEntry[String] = conf( - s"$COMET_EXEC_CONFIG_PREFIX.shuffle.codec") - .doc( - "The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported.") - .stringConf - .createWithDefault("zstd") + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") + .doc( + "The codec of Comet native shuffle used to compress shuffle data. lz4, zstd, and " + + "snappy are supported. Compression can be disabled by setting " + + "spark.shuffle.compress=false.") + .stringConf + .checkValues(Set("zstd", "lz4", "snappy")) + .createWithDefault("lz4") + + val COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL: ConfigEntry[Int] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.zstd.level") + .doc("The compression level to use when compressing shuffle files with zstd.") + .intConf + .createWithDefault(1) + + val COMET_SHUFFLE_ENABLE_FAST_ENCODING: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.enableFastEncoding") + .doc("Whether to enable Comet's faster proprietary encoding for shuffle blocks " + + "rather than using Arrow IPC.") + .internal() + .booleanConf + .createWithDefault(true) val COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.columnar.shuffle.async.enabled") @@ -465,21 +491,21 @@ object CometConf extends ShimCometConf { .intConf .createWithDefault(8192) - val COMET_EXEC_MEMORY_FRACTION: ConfigEntry[Double] = conf("spark.comet.exec.memoryFraction") - .doc( - "The fraction of memory from Comet memory overhead that the native memory " + - "manager can use for execution. The purpose of this config is to set aside memory for " + - "untracked data structures, as well as imprecise size estimation during memory " + - "acquisition.") - .doubleConf - .createWithDefault(0.7) - val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = conf("spark.comet.parquet.enable.directBuffer") .doc("Whether to use Java direct byte buffer when reading Parquet.") .booleanConf .createWithDefault(false) + val COMET_EXEC_MEMORY_POOL_TYPE: ConfigEntry[String] = conf("spark.comet.exec.memoryPool") + .doc( + "The type of memory pool to be used for Comet native execution. " + + "Available memory pool types are 'greedy', 'fair_spill', 'greedy_task_shared', " + + "'fair_spill_task_shared', 'greedy_global' and 'fair_spill_global', By default, " + + "this config is 'greedy_task_shared'.") + .stringConf + .createWithDefault("greedy_task_shared") + val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.preFetch.enabled") .doc("Whether to enable pre-fetching feature of CometScan.") diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index 72472a5401..fba4e29e5e 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -163,8 +163,6 @@ class NativeUtil { case numRows => val cometVectors = importVector(arrays, schemas) Some(new ColumnarBatch(cometVectors.toArray, numRows.toInt)) - case flag => - throw new IllegalStateException(s"Invalid native flag: $flag") } } diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala b/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala deleted file mode 100644 index 281c481083..0000000000 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/IpcInputStreamIterator.scala +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.comet.execution.shuffle - -import java.io.EOFException -import java.io.InputStream -import java.nio.ByteBuffer -import java.nio.ByteOrder -import java.nio.channels.Channels -import java.nio.channels.ReadableByteChannel - -import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging -import org.apache.spark.network.util.LimitedInputStream - -case class IpcInputStreamIterator( - var in: InputStream, - decompressingNeeded: Boolean, - taskContext: TaskContext) - extends Iterator[ReadableByteChannel] - with Logging { - - private[execution] val channel: ReadableByteChannel = if (in != null) { - Channels.newChannel(in) - } else { - null - } - - private val ipcLengthsBuf = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN) - - // NOTE: - // since all ipcs are sharing the same input stream and channel, the second - // hasNext() must be called after the first ipc has been completely processed. - - private[execution] var consumed = true - private var finished = false - private var currentIpcLength = 0L - private var currentLimitedInputStream: LimitedInputStream = _ - - taskContext.addTaskCompletionListener[Unit](_ => { - closeInputStream() - }) - - override def hasNext: Boolean = { - if (in == null || finished) { - return false - } - - // If we've read the length of the next IPC, we don't need to read it again. - if (!consumed) { - return true - } - - if (currentLimitedInputStream != null) { - currentLimitedInputStream.skip(Int.MaxValue) - currentLimitedInputStream = null - } - - // Reads the length of IPC bytes - ipcLengthsBuf.clear() - while (ipcLengthsBuf.hasRemaining && channel.read(ipcLengthsBuf) >= 0) {} - - // If we reach the end of the stream, we are done, or if we read partial length - // then the stream is corrupted. - if (ipcLengthsBuf.hasRemaining) { - if (ipcLengthsBuf.position() == 0) { - finished = true - closeInputStream() - return false - } - throw new EOFException("Data corrupt: unexpected EOF while reading compressed ipc lengths") - } - - ipcLengthsBuf.flip() - currentIpcLength = ipcLengthsBuf.getLong - - // Skips empty IPC - if (currentIpcLength == 0) { - return hasNext - } - consumed = false - return true - } - - override def next(): ReadableByteChannel = { - if (!hasNext) { - throw new NoSuchElementException - } - assert(!consumed) - consumed = true - - val is = new LimitedInputStream(Channels.newInputStream(channel), currentIpcLength, false) - currentLimitedInputStream = is - - if (decompressingNeeded) { - val zs = ShuffleUtils.compressionCodecForShuffling.compressedInputStream(is) - Channels.newChannel(zs) - } else { - Channels.newChannel(is) - } - } - - private def closeInputStream(): Unit = - synchronized { - if (in != null) { - in.close() - in = null - } - } -} diff --git a/dev/changelog/0.5.0.md b/dev/changelog/0.5.0.md new file mode 100644 index 0000000000..af2ae1cfd8 --- /dev/null +++ b/dev/changelog/0.5.0.md @@ -0,0 +1,129 @@ + + +# DataFusion Comet 0.5.0 Changelog + +This release consists of 69 commits from 15 contributors. See credits at the end of this changelog for more information. + +**Fixed bugs:** + +- fix: Unsigned type related bugs [#1095](https://github.com/apache/datafusion-comet/pull/1095) (kazuyukitanimura) +- fix: Use RDD partition index [#1112](https://github.com/apache/datafusion-comet/pull/1112) (viirya) +- fix: Various metrics bug fixes and improvements [#1111](https://github.com/apache/datafusion-comet/pull/1111) (andygrove) +- fix: Don't create CometScanExec for subclasses of ParquetFileFormat [#1129](https://github.com/apache/datafusion-comet/pull/1129) (Kimahriman) +- fix: Fix metrics regressions [#1132](https://github.com/apache/datafusion-comet/pull/1132) (andygrove) +- fix: Enable scenarios accidentally commented out in CometExecBenchmark [#1151](https://github.com/apache/datafusion-comet/pull/1151) (mbutrovich) +- fix: Spark 4.0-preview1 SPARK-47120 [#1156](https://github.com/apache/datafusion-comet/pull/1156) (kazuyukitanimura) +- fix: Document enabling comet explain plan usage in Spark (4.0) [#1176](https://github.com/apache/datafusion-comet/pull/1176) (parthchandra) +- fix: stddev_pop should not directly return 0.0 when count is 1.0 [#1184](https://github.com/apache/datafusion-comet/pull/1184) (viirya) +- fix: fix missing explanation for then branch in case when [#1200](https://github.com/apache/datafusion-comet/pull/1200) (rluvaton) +- fix: Fall back to Spark for unsupported partition or sort expressions in window aggregates [#1253](https://github.com/apache/datafusion-comet/pull/1253) (andygrove) +- fix: Fall back to Spark for distinct aggregates [#1262](https://github.com/apache/datafusion-comet/pull/1262) (andygrove) +- fix: disable initCap by default [#1276](https://github.com/apache/datafusion-comet/pull/1276) (kazuyukitanimura) + +**Performance related:** + +- perf: Stop passing Java config map into native createPlan [#1101](https://github.com/apache/datafusion-comet/pull/1101) (andygrove) +- feat: Make native shuffle compression configurable and respect `spark.shuffle.compress` [#1185](https://github.com/apache/datafusion-comet/pull/1185) (andygrove) +- perf: Improve query planning to more reliably fall back to columnar shuffle when native shuffle is not supported [#1209](https://github.com/apache/datafusion-comet/pull/1209) (andygrove) +- feat: Move shuffle block decompression and decoding to native code and add LZ4 & Snappy support [#1192](https://github.com/apache/datafusion-comet/pull/1192) (andygrove) +- feat: Implement custom RecordBatch serde for shuffle for improved performance [#1190](https://github.com/apache/datafusion-comet/pull/1190) (andygrove) + +**Implemented enhancements:** + +- feat: support array_insert [#1073](https://github.com/apache/datafusion-comet/pull/1073) (SemyonSinchenko) +- feat: enable decimal to decimal cast of different precision and scale [#1086](https://github.com/apache/datafusion-comet/pull/1086) (himadripal) +- feat: Improve ScanExec native metrics [#1133](https://github.com/apache/datafusion-comet/pull/1133) (andygrove) +- feat: Add Spark-compatible implementation of SchemaAdapterFactory [#1169](https://github.com/apache/datafusion-comet/pull/1169) (andygrove) +- feat: Improve shuffle metrics (second attempt) [#1175](https://github.com/apache/datafusion-comet/pull/1175) (andygrove) +- feat: Add a `spark.comet.exec.memoryPool` configuration for experimenting with various datafusion memory pool setups. [#1021](https://github.com/apache/datafusion-comet/pull/1021) (Kontinuation) +- feat: Reenable tests for filtered SMJ anti join [#1211](https://github.com/apache/datafusion-comet/pull/1211) (comphead) +- feat: add support for array_remove expression [#1179](https://github.com/apache/datafusion-comet/pull/1179) (jatin510) + +**Documentation updates:** + +- docs: Update documentation for 0.4.0 release [#1096](https://github.com/apache/datafusion-comet/pull/1096) (andygrove) +- docs: Fix readme typo FGPA -> FPGA [#1117](https://github.com/apache/datafusion-comet/pull/1117) (gstvg) +- docs: Add more technical detail and new diagram to Comet plugin overview [#1119](https://github.com/apache/datafusion-comet/pull/1119) (andygrove) +- docs: Add some documentation explaining how shuffle works [#1148](https://github.com/apache/datafusion-comet/pull/1148) (andygrove) +- docs: Update TPC-H benchmark results [#1257](https://github.com/apache/datafusion-comet/pull/1257) (andygrove) + +**Other:** + +- chore: Add changelog for 0.4.0 [#1089](https://github.com/apache/datafusion-comet/pull/1089) (andygrove) +- chore: Prepare for 0.5.0 development [#1090](https://github.com/apache/datafusion-comet/pull/1090) (andygrove) +- build: Skip installation of spark-integration and fuzz testing modules [#1091](https://github.com/apache/datafusion-comet/pull/1091) (parthchandra) +- minor: Add hint for finding the GPG key to use when publishing to maven [#1093](https://github.com/apache/datafusion-comet/pull/1093) (andygrove) +- chore: Include first ScanExec batch in metrics [#1105](https://github.com/apache/datafusion-comet/pull/1105) (andygrove) +- chore: Improve CometScan metrics [#1100](https://github.com/apache/datafusion-comet/pull/1100) (andygrove) +- chore: Add custom metric for native shuffle fetching batches from JVM [#1108](https://github.com/apache/datafusion-comet/pull/1108) (andygrove) +- chore: Remove unused StringView struct [#1143](https://github.com/apache/datafusion-comet/pull/1143) (andygrove) +- test: enable more Spark 4.0 tests [#1145](https://github.com/apache/datafusion-comet/pull/1145) (kazuyukitanimura) +- chore: Refactor cast to use SparkCastOptions param [#1146](https://github.com/apache/datafusion-comet/pull/1146) (andygrove) +- chore: Move more expressions from core crate to spark-expr crate [#1152](https://github.com/apache/datafusion-comet/pull/1152) (andygrove) +- chore: Remove dead code [#1155](https://github.com/apache/datafusion-comet/pull/1155) (andygrove) +- chore: Move string kernels and expressions to spark-expr crate [#1164](https://github.com/apache/datafusion-comet/pull/1164) (andygrove) +- chore: Move remaining expressions to spark-expr crate + some minor refactoring [#1165](https://github.com/apache/datafusion-comet/pull/1165) (andygrove) +- chore: Add ignored tests for reading complex types from Parquet [#1167](https://github.com/apache/datafusion-comet/pull/1167) (andygrove) +- test: enabling Spark tests with offHeap requirement [#1177](https://github.com/apache/datafusion-comet/pull/1177) (kazuyukitanimura) +- minor: move shuffle classes from common to spark [#1193](https://github.com/apache/datafusion-comet/pull/1193) (andygrove) +- minor: refactor to move decodeBatches to broadcast exchange code as private function [#1195](https://github.com/apache/datafusion-comet/pull/1195) (andygrove) +- minor: refactor prepare_output so that it does not require an ExecutionContext [#1194](https://github.com/apache/datafusion-comet/pull/1194) (andygrove) +- minor: remove unused source files [#1202](https://github.com/apache/datafusion-comet/pull/1202) (andygrove) +- chore: Upgrade to DataFusion 44.0.0-rc2 [#1154](https://github.com/apache/datafusion-comet/pull/1154) (andygrove) +- chore: Add safety check to CometBuffer [#1050](https://github.com/apache/datafusion-comet/pull/1050) (viirya) +- chore: Remove unreachable code [#1213](https://github.com/apache/datafusion-comet/pull/1213) (andygrove) +- test: Enable Comet by default except some tests in SparkSessionExtensionSuite [#1201](https://github.com/apache/datafusion-comet/pull/1201) (kazuyukitanimura) +- chore: extract `struct` expressions to folders based on spark grouping [#1216](https://github.com/apache/datafusion-comet/pull/1216) (rluvaton) +- chore: extract static invoke expressions to folders based on spark grouping [#1217](https://github.com/apache/datafusion-comet/pull/1217) (rluvaton) +- chore: Follow-on PR to fully enable onheap memory usage [#1210](https://github.com/apache/datafusion-comet/pull/1210) (andygrove) +- chore: extract agg_funcs expressions to folders based on spark grouping [#1224](https://github.com/apache/datafusion-comet/pull/1224) (rluvaton) +- chore: extract datetime_funcs expressions to folders based on spark grouping [#1222](https://github.com/apache/datafusion-comet/pull/1222) (rluvaton) +- chore: Upgrade to DataFusion 44.0.0 from 44.0.0 RC2 [#1232](https://github.com/apache/datafusion-comet/pull/1232) (rluvaton) +- chore: extract strings file to `strings_func` like in spark grouping [#1215](https://github.com/apache/datafusion-comet/pull/1215) (rluvaton) +- chore: extract predicate_functions expressions to folders based on spark grouping [#1218](https://github.com/apache/datafusion-comet/pull/1218) (rluvaton) +- build(deps): bump protobuf version to 3.21.12 [#1234](https://github.com/apache/datafusion-comet/pull/1234) (wForget) +- chore: extract json_funcs expressions to folders based on spark grouping [#1220](https://github.com/apache/datafusion-comet/pull/1220) (rluvaton) +- test: Enable shuffle by default in Spark tests [#1240](https://github.com/apache/datafusion-comet/pull/1240) (kazuyukitanimura) +- chore: extract hash_funcs expressions to folders based on spark grouping [#1221](https://github.com/apache/datafusion-comet/pull/1221) (rluvaton) +- build: Fix test failure caused by merging conflicting PRs [#1259](https://github.com/apache/datafusion-comet/pull/1259) (andygrove) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 37 Andy Grove + 10 Raz Luvaton + 7 KAZUYUKI TANIMURA + 3 Liang-Chi Hsieh + 2 Parth Chandra + 1 Adam Binford + 1 Dharan Aditya + 1 Himadri Pal + 1 Jagdish Parihar + 1 Kristin Cowalcijk + 1 Matt Butrovich + 1 Oleks V + 1 Sem + 1 Zhen Wang + 1 gstvg +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 12f739848c..ed1dfa21cf 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index d3544881af1..bf0e2b53c70 100644 +index d3544881af1..26ab186c65d 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -38,7 +38,7 @@ index d3544881af1..bf0e2b53c70 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index b386d135da1..854aec17c2d 100644 +index b386d135da1..46449e3f3f1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -53,7 +53,7 @@ index b386d135da1..854aec17c2d 100644 {})", + self.input_data_type, + self.result_data_type + ), + } + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "sum"), + self.input_data_type.clone(), + true, + ), + Field::new( + format_state_name(&self.name, "count"), + DataType::Int64, + true, + ), + ]) + } + + fn name(&self) -> &str { + &self.name + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + // instantiate specialized accumulator based for the type + match (&self.input_data_type, &self.result_data_type) { + (Float64, Float64) => Ok(Box::new(AvgGroupsAccumulator::::new( + &self.input_data_type, + |sum: f64, count: i64| Ok(sum / count as f64), + ))), + + _ => not_impl_err!( + "AvgGroupsAccumulator for ({} --> {})", + self.input_data_type, + self.result_data_type + ), + } + } + + fn default_value(&self, _data_type: &DataType) -> Result { + Ok(ScalarValue::Float64(None)) + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + avg_return_type(self.name(), &arg_types[0]) + } +} + +/// An accumulator to compute the average +#[derive(Debug, Default)] +pub struct AvgAccumulator { + sum: Option, + count: i64, +} + +impl Accumulator for AvgAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::Float64(self.sum), + ScalarValue::from(self.count), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = values[0].as_primitive::(); + self.count += (values.len() - values.null_count()) as i64; + let v = self.sum.get_or_insert(0.); + if let Some(x) = sum(values) { + *v += x; + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // counts are summed + self.count += sum(states[1].as_primitive::()).unwrap_or_default(); + + // sums are summed + if let Some(x) = sum(states[0].as_primitive::()) { + let v = self.sum.get_or_insert(0.); + *v += x; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + if self.count == 0 { + // If all input are nulls, count will be 0 and we will get null after the division. + // This is consistent with Spark Average implementation. + Ok(ScalarValue::Float64(None)) + } else { + Ok(ScalarValue::Float64( + self.sum.map(|f| f / self.count as f64), + )) + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} + +/// An accumulator to compute the average of `[PrimitiveArray]`. +/// Stores values as native types, and does overflow checking +/// +/// F: Function that calculates the average value from a sum of +/// T::Native and a total count +#[derive(Debug)] +struct AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result + Send, +{ + /// The type of the returned average + return_data_type: DataType, + + /// Count per group (use i64 to make Int64Array) + counts: Vec, + + /// Sums per group, stored as the native type + sums: Vec, + + /// Function that computes the final average (value / count) + avg_fn: F, +} + +impl AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result + Send, +{ + pub fn new(return_data_type: &DataType, avg_fn: F) -> Self { + Self { + return_data_type: return_data_type.clone(), + counts: vec![], + sums: vec![], + avg_fn, + } + } +} + +impl GroupsAccumulator for AvgGroupsAccumulator +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result + Send, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + let data = values.values(); + + // increment counts, update sums + self.counts.resize(total_num_groups, 0); + self.sums.resize(total_num_groups, T::default_value()); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + let sum = &mut self.sums[group_index]; + *sum = (*sum).add_wrapping(value); + self.counts[group_index] += 1; + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + let sum = &mut self.sums[group_index]; + *sum = (*sum).add_wrapping(value); + + self.counts[group_index] += 1; + } + } + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is partial sums, second is counts + let partial_sums = values[0].as_primitive::(); + let partial_counts = values[1].as_primitive::(); + // update counts with partial counts + self.counts.resize(total_num_groups, 0); + let iter1 = group_indices.iter().zip(partial_counts.values().iter()); + for (&group_index, &partial_count) in iter1 { + self.counts[group_index] += partial_count; + } + + // update sums + self.sums.resize(total_num_groups, T::default_value()); + let iter2 = group_indices.iter().zip(partial_sums.values().iter()); + for (&group_index, &new_value) in iter2 { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let counts = emit_to.take_needed(&mut self.counts); + let sums = emit_to.take_needed(&mut self.sums); + let mut builder = PrimitiveBuilder::::with_capacity(sums.len()); + let iter = sums.into_iter().zip(counts); + + for (sum, count) in iter { + if count != 0 { + builder.append_value((self.avg_fn)(sum, count)?) + } else { + builder.append_null(); + } + } + let array: PrimitiveArray = builder.finish(); + + Ok(Arc::new(array)) + } + + // return arrays for sums and counts + fn state(&mut self, emit_to: EmitTo) -> Result> { + let counts = emit_to.take_needed(&mut self.counts); + let counts = Int64Array::new(counts.into(), None); + + let sums = emit_to.take_needed(&mut self.sums); + let sums = PrimitiveArray::::new(sums.into(), None) + .with_data_type(self.return_data_type.clone()); + + Ok(vec![ + Arc::new(sums) as ArrayRef, + Arc::new(counts) as ArrayRef, + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::() + + self.sums.capacity() * std::mem::size_of::() + } +} diff --git a/native/spark-expr/src/agg_funcs/avg_decimal.rs b/native/spark-expr/src/agg_funcs/avg_decimal.rs new file mode 100644 index 0000000000..05fc28e583 --- /dev/null +++ b/native/spark-expr/src/agg_funcs/avg_decimal.rs @@ -0,0 +1,522 @@ +// 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. + +use arrow::{array::BooleanBufferBuilder, buffer::NullBuffer, compute::sum}; +use arrow_array::{ + builder::PrimitiveBuilder, + cast::AsArray, + types::{Decimal128Type, Int64Type}, + Array, ArrayRef, Decimal128Array, Int64Array, PrimitiveArray, +}; +use arrow_schema::{DataType, Field}; +use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator, Signature}; +use datafusion_common::{not_impl_err, Result, ScalarValue}; +use datafusion_physical_expr::expressions::format_state_name; +use std::{any::Any, sync::Arc}; + +use crate::utils::is_valid_decimal_precision; +use arrow_array::ArrowNativeTypeOp; +use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; +use datafusion::logical_expr::Volatility::Immutable; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::type_coercion::aggregates::avg_return_type; +use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; +use num::{integer::div_ceil, Integer}; +use DataType::*; + +/// AVG aggregate expression +#[derive(Debug, Clone)] +pub struct AvgDecimal { + signature: Signature, + sum_data_type: DataType, + result_data_type: DataType, +} + +impl AvgDecimal { + /// Create a new AVG aggregate function + pub fn new(result_type: DataType, sum_type: DataType) -> Self { + Self { + signature: Signature::user_defined(Immutable), + result_data_type: result_type, + sum_data_type: sum_type, + } + } +} + +impl AggregateUDFImpl for AvgDecimal { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { + match (&self.sum_data_type, &self.result_data_type) { + (Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale)) => { + Ok(Box::new(AvgDecimalAccumulator::new( + *sum_scale, + *sum_precision, + *target_precision, + *target_scale, + ))) + } + _ => not_impl_err!( + "AvgDecimalAccumulator for ({} --> {})", + self.sum_data_type, + self.result_data_type + ), + } + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new( + format_state_name(self.name(), "sum"), + self.sum_data_type.clone(), + true, + ), + Field::new( + format_state_name(self.name(), "count"), + DataType::Int64, + true, + ), + ]) + } + + fn name(&self) -> &str { + "avg" + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + // instantiate specialized accumulator based for the type + match (&self.sum_data_type, &self.result_data_type) { + (Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale)) => { + Ok(Box::new(AvgDecimalGroupsAccumulator::new( + &self.result_data_type, + &self.sum_data_type, + *target_precision, + *target_scale, + *sum_precision, + *sum_scale, + ))) + } + _ => not_impl_err!( + "AvgDecimalGroupsAccumulator for ({} --> {})", + self.sum_data_type, + self.result_data_type + ), + } + } + + fn default_value(&self, _data_type: &DataType) -> Result { + match &self.result_data_type { + Decimal128(target_precision, target_scale) => { + Ok(make_decimal128(None, *target_precision, *target_scale)) + } + _ => not_impl_err!( + "The result_data_type of AvgDecimal should be Decimal128 but got{}", + self.result_data_type + ), + } + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + avg_return_type(self.name(), &arg_types[0]) + } +} + +/// An accumulator to compute the average for decimals +#[derive(Debug)] +struct AvgDecimalAccumulator { + sum: Option, + count: i64, + is_empty: bool, + is_not_null: bool, + sum_scale: i8, + sum_precision: u8, + target_precision: u8, + target_scale: i8, +} + +impl AvgDecimalAccumulator { + pub fn new(sum_scale: i8, sum_precision: u8, target_precision: u8, target_scale: i8) -> Self { + Self { + sum: None, + count: 0, + is_empty: true, + is_not_null: true, + sum_scale, + sum_precision, + target_precision, + target_scale, + } + } + + fn update_single(&mut self, values: &Decimal128Array, idx: usize) { + let v = unsafe { values.value_unchecked(idx) }; + let (new_sum, is_overflow) = match self.sum { + Some(sum) => sum.overflowing_add(v), + None => (v, false), + }; + + if is_overflow || !is_valid_decimal_precision(new_sum, self.sum_precision) { + // Overflow: set buffer accumulator to null + self.is_not_null = false; + return; + } + + self.sum = Some(new_sum); + + if let Some(new_count) = self.count.checked_add(1) { + self.count = new_count; + } else { + self.is_not_null = false; + return; + } + + self.is_not_null = true; + } +} + +fn make_decimal128(value: Option, precision: u8, scale: i8) -> ScalarValue { + ScalarValue::Decimal128(value, precision, scale) +} + +impl Accumulator for AvgDecimalAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::Decimal128(self.sum, self.sum_precision, self.sum_scale), + ScalarValue::from(self.count), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if !self.is_empty && !self.is_not_null { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return Ok(()); + } + + let values = &values[0]; + let data = values.as_primitive::(); + + self.is_empty = self.is_empty && values.len() == values.null_count(); + + if values.null_count() == 0 { + for i in 0..data.len() { + self.update_single(data, i); + } + } else { + for i in 0..data.len() { + if data.is_null(i) { + continue; + } + self.update_single(data, i); + } + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // counts are summed + self.count += sum(states[1].as_primitive::()).unwrap_or_default(); + + // sums are summed + if let Some(x) = sum(states[0].as_primitive::()) { + let v = self.sum.get_or_insert(0); + let (result, overflowed) = v.overflowing_add(x); + if overflowed { + // Set to None if overflow happens + self.sum = None; + } else { + *v = result; + } + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let scaler = 10_i128.pow(self.target_scale.saturating_sub(self.sum_scale) as u32); + let target_min = MIN_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + let target_max = MAX_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + + let result = self + .sum + .map(|v| avg(v, self.count as i128, target_min, target_max, scaler)); + + match result { + Some(value) => Ok(make_decimal128( + value, + self.target_precision, + self.target_scale, + )), + _ => Ok(make_decimal128( + None, + self.target_precision, + self.target_scale, + )), + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} + +#[derive(Debug)] +struct AvgDecimalGroupsAccumulator { + /// Tracks if the value is null + is_not_null: BooleanBufferBuilder, + + // Tracks if the value is empty + is_empty: BooleanBufferBuilder, + + /// The type of the avg return type + return_data_type: DataType, + target_precision: u8, + target_scale: i8, + + /// Count per group (use i64 to make Int64Array) + counts: Vec, + + /// Sums per group, stored as i128 + sums: Vec, + + /// The type of the sum + sum_data_type: DataType, + /// This is input_precision + 10 to be consistent with Spark + sum_precision: u8, + sum_scale: i8, +} + +impl AvgDecimalGroupsAccumulator { + pub fn new( + return_data_type: &DataType, + sum_data_type: &DataType, + target_precision: u8, + target_scale: i8, + sum_precision: u8, + sum_scale: i8, + ) -> Self { + Self { + is_not_null: BooleanBufferBuilder::new(0), + is_empty: BooleanBufferBuilder::new(0), + return_data_type: return_data_type.clone(), + target_precision, + target_scale, + sum_data_type: sum_data_type.clone(), + sum_precision, + sum_scale, + counts: vec![], + sums: vec![], + } + } + + fn is_overflow(&self, index: usize) -> bool { + !self.is_empty.get_bit(index) && !self.is_not_null.get_bit(index) + } + + fn update_single(&mut self, group_index: usize, value: i128) { + if self.is_overflow(group_index) { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return; + } + + self.is_empty.set_bit(group_index, false); + let (new_sum, is_overflow) = self.sums[group_index].overflowing_add(value); + self.counts[group_index] += 1; + + if is_overflow || !is_valid_decimal_precision(new_sum, self.sum_precision) { + // Overflow: set buffer accumulator to null + self.is_not_null.set_bit(group_index, false); + return; + } + + self.sums[group_index] = new_sum; + self.is_not_null.set_bit(group_index, true) + } +} + +fn ensure_bit_capacity(builder: &mut BooleanBufferBuilder, capacity: usize) { + if builder.len() < capacity { + let additional = capacity - builder.len(); + builder.append_n(additional, true); + } +} + +impl GroupsAccumulator for AvgDecimalGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + let data = values.values(); + + // increment counts, update sums + self.counts.resize(total_num_groups, 0); + self.sums.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + self.update_single(group_index, value); + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + self.update_single(group_index, value); + } + } + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is partial sums, second is counts + let partial_sums = values[0].as_primitive::(); + let partial_counts = values[1].as_primitive::(); + // update counts with partial counts + self.counts.resize(total_num_groups, 0); + let iter1 = group_indices.iter().zip(partial_counts.values().iter()); + for (&group_index, &partial_count) in iter1 { + self.counts[group_index] += partial_count; + } + + // update sums + self.sums.resize(total_num_groups, 0); + let iter2 = group_indices.iter().zip(partial_sums.values().iter()); + for (&group_index, &new_value) in iter2 { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let counts = emit_to.take_needed(&mut self.counts); + let sums = emit_to.take_needed(&mut self.sums); + + let mut builder = PrimitiveBuilder::::with_capacity(sums.len()) + .with_data_type(self.return_data_type.clone()); + let iter = sums.into_iter().zip(counts); + + let scaler = 10_i128.pow(self.target_scale.saturating_sub(self.sum_scale) as u32); + let target_min = MIN_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + let target_max = MAX_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + + for (sum, count) in iter { + if count != 0 { + match avg(sum, count as i128, target_min, target_max, scaler) { + Some(value) => { + builder.append_value(value); + } + _ => { + builder.append_null(); + } + } + } else { + builder.append_null(); + } + } + let array: PrimitiveArray = builder.finish(); + + Ok(Arc::new(array)) + } + + // return arrays for sums and counts + fn state(&mut self, emit_to: EmitTo) -> Result> { + let nulls = self.is_not_null.finish(); + let nulls = Some(NullBuffer::new(nulls)); + + let counts = emit_to.take_needed(&mut self.counts); + let counts = Int64Array::new(counts.into(), nulls.clone()); + + let sums = emit_to.take_needed(&mut self.sums); + let sums = + Decimal128Array::new(sums.into(), nulls).with_data_type(self.sum_data_type.clone()); + + Ok(vec![ + Arc::new(sums) as ArrayRef, + Arc::new(counts) as ArrayRef, + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::() + + self.sums.capacity() * std::mem::size_of::() + } +} + +/// Returns the `sum`/`count` as a i128 Decimal128 with +/// target_scale and target_precision and return None if overflows. +/// +/// * sum: The total sum value stored as Decimal128 with sum_scale +/// * count: total count, stored as a i128 (*NOT* a Decimal128 value) +/// * target_min: The minimum output value possible to represent with the target precision +/// * target_max: The maximum output value possible to represent with the target precision +/// * scaler: scale factor for avg +#[inline(always)] +fn avg(sum: i128, count: i128, target_min: i128, target_max: i128, scaler: i128) -> Option { + if let Some(value) = sum.checked_mul(scaler) { + // `sum / count` with ROUND_HALF_UP + let (div, rem) = value.div_rem(&count); + let half = div_ceil(count, 2); + let half_neg = half.neg_wrapping(); + let new_value = match value >= 0 { + true if rem >= half => div.add_wrapping(1), + false if rem <= half_neg => div.sub_wrapping(1), + _ => div, + }; + if new_value >= target_min && new_value <= target_max { + Some(new_value) + } else { + None + } + } else { + None + } +} diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/agg_funcs/correlation.rs similarity index 98% rename from native/spark-expr/src/correlation.rs rename to native/spark-expr/src/agg_funcs/correlation.rs index e4ddab95de..5d6f9e0b43 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/agg_funcs/correlation.rs @@ -19,8 +19,8 @@ use arrow::compute::{and, filter, is_not_null}; use std::{any::Any, sync::Arc}; -use crate::covariance::CovarianceAccumulator; -use crate::stddev::StddevAccumulator; +use crate::agg_funcs::covariance::CovarianceAccumulator; +use crate::agg_funcs::stddev::StddevAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/spark-expr/src/agg_funcs/covariance.rs b/native/spark-expr/src/agg_funcs/covariance.rs new file mode 100644 index 0000000000..fa3563cdea --- /dev/null +++ b/native/spark-expr/src/agg_funcs/covariance.rs @@ -0,0 +1,306 @@ +/* + * 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. + */ + +use std::any::Any; + +use arrow::{ + array::{ArrayRef, Float64Array}, + compute::cast, + datatypes::{DataType, Field}, +}; +use datafusion::logical_expr::Accumulator; +use datafusion_common::{ + downcast_value, unwrap_or_internal_err, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::type_coercion::aggregates::NUMERICS; +use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; +use datafusion_physical_expr::expressions::StatsType; + +/// COVAR_SAMP and COVAR_POP aggregate expression +/// The implementation mostly is the same as the DataFusion's implementation. The reason +/// we have our own implementation is that DataFusion has UInt64 for state_field count, +/// while Spark has Double for count. +#[derive(Debug, Clone)] +pub struct Covariance { + name: String, + signature: Signature, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl Covariance { + /// Create a new COVAR aggregate function + pub fn new( + name: impl Into, + data_type: DataType, + stats_type: StatsType, + null_on_divide_by_zero: bool, + ) -> Self { + // the result of covariance just support FLOAT64 data type. + assert!(matches!(data_type, DataType::Float64)); + Self { + name: name.into(), + signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), + stats_type, + null_on_divide_by_zero, + } + } +} + +impl AggregateUDFImpl for Covariance { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + fn default_value(&self, _data_type: &DataType) -> Result { + Ok(ScalarValue::Float64(None)) + } + + fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { + Ok(Box::new(CovarianceAccumulator::try_new( + self.stats_type, + self.null_on_divide_by_zero, + )?)) + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "count"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "mean1"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "mean2"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "algo_const"), + DataType::Float64, + true, + ), + ]) + } +} + +/// An accumulator to compute covariance +#[derive(Debug)] +pub struct CovarianceAccumulator { + algo_const: f64, + mean1: f64, + mean2: f64, + count: f64, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl CovarianceAccumulator { + /// Creates a new `CovarianceAccumulator` + pub fn try_new(s_type: StatsType, null_on_divide_by_zero: bool) -> Result { + Ok(Self { + algo_const: 0_f64, + mean1: 0_f64, + mean2: 0_f64, + count: 0_f64, + stats_type: s_type, + null_on_divide_by_zero, + }) + } + + pub fn get_count(&self) -> f64 { + self.count + } + + pub fn get_mean1(&self) -> f64 { + self.mean1 + } + + pub fn get_mean2(&self) -> f64 { + self.mean2 + } + + pub fn get_algo_const(&self) -> f64 { + self.algo_const + } +} + +impl Accumulator for CovarianceAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::from(self.count), + ScalarValue::from(self.mean1), + ScalarValue::from(self.mean2), + ScalarValue::from(self.algo_const), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values1 = &cast(&values[0], &DataType::Float64)?; + let values2 = &cast(&values[1], &DataType::Float64)?; + + let mut arr1 = downcast_value!(values1, Float64Array).iter().flatten(); + let mut arr2 = downcast_value!(values2, Float64Array).iter().flatten(); + + for i in 0..values1.len() { + let value1 = if values1.is_valid(i) { + arr1.next() + } else { + None + }; + let value2 = if values2.is_valid(i) { + arr2.next() + } else { + None + }; + + if value1.is_none() || value2.is_none() { + continue; + } + + let value1 = unwrap_or_internal_err!(value1); + let value2 = unwrap_or_internal_err!(value2); + let new_count = self.count + 1.0; + let delta1 = value1 - self.mean1; + let new_mean1 = delta1 / new_count + self.mean1; + let delta2 = value2 - self.mean2; + let new_mean2 = delta2 / new_count + self.mean2; + let new_c = delta1 * (value2 - new_mean2) + self.algo_const; + + self.count += 1.0; + self.mean1 = new_mean1; + self.mean2 = new_mean2; + self.algo_const = new_c; + } + + Ok(()) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values1 = &cast(&values[0], &DataType::Float64)?; + let values2 = &cast(&values[1], &DataType::Float64)?; + let mut arr1 = downcast_value!(values1, Float64Array).iter().flatten(); + let mut arr2 = downcast_value!(values2, Float64Array).iter().flatten(); + + for i in 0..values1.len() { + let value1 = if values1.is_valid(i) { + arr1.next() + } else { + None + }; + let value2 = if values2.is_valid(i) { + arr2.next() + } else { + None + }; + + if value1.is_none() || value2.is_none() { + continue; + } + + let value1 = unwrap_or_internal_err!(value1); + let value2 = unwrap_or_internal_err!(value2); + + let new_count = self.count - 1.0; + let delta1 = self.mean1 - value1; + let new_mean1 = delta1 / new_count + self.mean1; + let delta2 = self.mean2 - value2; + let new_mean2 = delta2 / new_count + self.mean2; + let new_c = self.algo_const - delta1 * (new_mean2 - value2); + + self.count -= 1.0; + self.mean1 = new_mean1; + self.mean2 = new_mean2; + self.algo_const = new_c; + } + + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let counts = downcast_value!(states[0], Float64Array); + let means1 = downcast_value!(states[1], Float64Array); + let means2 = downcast_value!(states[2], Float64Array); + let cs = downcast_value!(states[3], Float64Array); + + for i in 0..counts.len() { + let c = counts.value(i); + if c == 0.0 { + continue; + } + let new_count = self.count + c; + let new_mean1 = self.mean1 * self.count / new_count + means1.value(i) * c / new_count; + let new_mean2 = self.mean2 * self.count / new_count + means2.value(i) * c / new_count; + let delta1 = self.mean1 - means1.value(i); + let delta2 = self.mean2 - means2.value(i); + let new_c = + self.algo_const + cs.value(i) + delta1 * delta2 * self.count * c / new_count; + + self.count = new_count; + self.mean1 = new_mean1; + self.mean2 = new_mean2; + self.algo_const = new_c; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + if self.count == 0.0 { + return Ok(ScalarValue::Float64(None)); + } + + let count = match self.stats_type { + StatsType::Population => self.count, + StatsType::Sample if self.count > 1.0 => self.count - 1.0, + StatsType::Sample => { + // self.count == 1.0 + return if self.null_on_divide_by_zero { + Ok(ScalarValue::Float64(None)) + } else { + Ok(ScalarValue::Float64(Some(f64::NAN))) + }; + } + }; + + Ok(ScalarValue::Float64(Some(self.algo_const / count))) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} diff --git a/native/spark-expr/src/agg_funcs/mod.rs b/native/spark-expr/src/agg_funcs/mod.rs new file mode 100644 index 0000000000..252da78890 --- /dev/null +++ b/native/spark-expr/src/agg_funcs/mod.rs @@ -0,0 +1,32 @@ +// 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. + +mod avg; +mod avg_decimal; +mod correlation; +mod covariance; +mod stddev; +mod sum_decimal; +mod variance; + +pub use avg::Avg; +pub use avg_decimal::AvgDecimal; +pub use correlation::Correlation; +pub use covariance::Covariance; +pub use stddev::Stddev; +pub use sum_decimal::SumDecimal; +pub use variance::Variance; diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/agg_funcs/stddev.rs similarity index 99% rename from native/spark-expr/src/stddev.rs rename to native/spark-expr/src/agg_funcs/stddev.rs index 1ec5ffb69a..39dffa1c8e 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/agg_funcs/stddev.rs @@ -17,7 +17,7 @@ use std::{any::Any, sync::Arc}; -use crate::variance::VarianceAccumulator; +use crate::agg_funcs::variance::VarianceAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/spark-expr/src/agg_funcs/sum_decimal.rs b/native/spark-expr/src/agg_funcs/sum_decimal.rs new file mode 100644 index 0000000000..f3f34d9bfa --- /dev/null +++ b/native/spark-expr/src/agg_funcs/sum_decimal.rs @@ -0,0 +1,555 @@ +// 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. + +use crate::utils::{is_valid_decimal_precision, unlikely}; +use arrow::{ + array::BooleanBufferBuilder, + buffer::{BooleanBuffer, NullBuffer}, +}; +use arrow_array::{ + cast::AsArray, types::Decimal128Type, Array, ArrayRef, BooleanArray, Decimal128Array, +}; +use arrow_schema::{DataType, Field}; +use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; +use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::Volatility::Immutable; +use datafusion_expr::{AggregateUDFImpl, ReversedUDAF, Signature}; +use std::{any::Any, ops::BitAnd, sync::Arc}; + +#[derive(Debug)] +pub struct SumDecimal { + /// Aggregate function signature + signature: Signature, + /// The data type of the SUM result. This will always be a decimal type + /// with the same precision and scale as specified in this struct + result_type: DataType, + /// Decimal precision + precision: u8, + /// Decimal scale + scale: i8, +} + +impl SumDecimal { + pub fn try_new(data_type: DataType) -> DFResult { + // The `data_type` is the SUM result type passed from Spark side + let (precision, scale) = match data_type { + DataType::Decimal128(p, s) => (p, s), + _ => { + return Err(DataFusionError::Internal( + "Invalid data type for SumDecimal".into(), + )) + } + }; + Ok(Self { + signature: Signature::user_defined(Immutable), + result_type: data_type, + precision, + scale, + }) + } +} + +impl AggregateUDFImpl for SumDecimal { + fn as_any(&self) -> &dyn Any { + self + } + + fn accumulator(&self, _args: AccumulatorArgs) -> DFResult> { + Ok(Box::new(SumDecimalAccumulator::new( + self.precision, + self.scale, + ))) + } + + fn state_fields(&self, _args: StateFieldsArgs) -> DFResult> { + let fields = vec![ + Field::new(self.name(), self.result_type.clone(), self.is_nullable()), + Field::new("is_empty", DataType::Boolean, false), + ]; + Ok(fields) + } + + fn name(&self) -> &str { + "sum" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> DFResult { + Ok(self.result_type.clone()) + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> DFResult> { + Ok(Box::new(SumDecimalGroupsAccumulator::new( + self.result_type.clone(), + self.precision, + ))) + } + + fn default_value(&self, _data_type: &DataType) -> DFResult { + ScalarValue::new_primitive::( + None, + &DataType::Decimal128(self.precision, self.scale), + ) + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } + + fn is_nullable(&self) -> bool { + // SumDecimal is always nullable because overflows can cause null values + true + } +} + +#[derive(Debug)] +struct SumDecimalAccumulator { + sum: i128, + is_empty: bool, + is_not_null: bool, + + precision: u8, + scale: i8, +} + +impl SumDecimalAccumulator { + fn new(precision: u8, scale: i8) -> Self { + Self { + sum: 0, + is_empty: true, + is_not_null: true, + precision, + scale, + } + } + + fn update_single(&mut self, values: &Decimal128Array, idx: usize) { + let v = unsafe { values.value_unchecked(idx) }; + let (new_sum, is_overflow) = self.sum.overflowing_add(v); + + if is_overflow || !is_valid_decimal_precision(new_sum, self.precision) { + // Overflow: set buffer accumulator to null + self.is_not_null = false; + return; + } + + self.sum = new_sum; + self.is_not_null = true; + } +} + +impl Accumulator for SumDecimalAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> DFResult<()> { + assert_eq!( + values.len(), + 1, + "Expect only one element in 'values' but found {}", + values.len() + ); + + if !self.is_empty && !self.is_not_null { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return Ok(()); + } + + let values = &values[0]; + let data = values.as_primitive::(); + + self.is_empty = self.is_empty && values.len() == values.null_count(); + + if values.null_count() == 0 { + for i in 0..data.len() { + self.update_single(data, i); + } + } else { + for i in 0..data.len() { + if data.is_null(i) { + continue; + } + self.update_single(data, i); + } + } + + Ok(()) + } + + fn evaluate(&mut self) -> DFResult { + // For each group: + // 1. if `is_empty` is true, it means either there is no value or all values for the group + // are null, in this case we'll return null + // 2. if `is_empty` is false, but `null_state` is true, it means there's an overflow. In + // non-ANSI mode Spark returns null. + if self.is_empty || !self.is_not_null { + ScalarValue::new_primitive::( + None, + &DataType::Decimal128(self.precision, self.scale), + ) + } else { + ScalarValue::try_new_decimal128(self.sum, self.precision, self.scale) + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> DFResult> { + let sum = if self.is_not_null { + ScalarValue::try_new_decimal128(self.sum, self.precision, self.scale)? + } else { + ScalarValue::new_primitive::( + None, + &DataType::Decimal128(self.precision, self.scale), + )? + }; + Ok(vec![sum, ScalarValue::from(self.is_empty)]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> DFResult<()> { + assert_eq!( + states.len(), + 2, + "Expect two element in 'states' but found {}", + states.len() + ); + assert_eq!(states[0].len(), 1); + assert_eq!(states[1].len(), 1); + + let that_sum = states[0].as_primitive::(); + let that_is_empty = states[1].as_any().downcast_ref::().unwrap(); + + let this_overflow = !self.is_empty && !self.is_not_null; + let that_overflow = !that_is_empty.value(0) && that_sum.is_null(0); + + self.is_not_null = !this_overflow && !that_overflow; + self.is_empty = self.is_empty && that_is_empty.value(0); + + if self.is_not_null { + self.sum += that_sum.value(0); + } + + Ok(()) + } +} + +struct SumDecimalGroupsAccumulator { + // Whether aggregate buffer for a particular group is null. True indicates it is not null. + is_not_null: BooleanBufferBuilder, + is_empty: BooleanBufferBuilder, + sum: Vec, + result_type: DataType, + precision: u8, +} + +impl SumDecimalGroupsAccumulator { + fn new(result_type: DataType, precision: u8) -> Self { + Self { + is_not_null: BooleanBufferBuilder::new(0), + is_empty: BooleanBufferBuilder::new(0), + sum: Vec::new(), + result_type, + precision, + } + } + + fn is_overflow(&self, index: usize) -> bool { + !self.is_empty.get_bit(index) && !self.is_not_null.get_bit(index) + } + + fn update_single(&mut self, group_index: usize, value: i128) { + if unlikely(self.is_overflow(group_index)) { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return; + } + + self.is_empty.set_bit(group_index, false); + let (new_sum, is_overflow) = self.sum[group_index].overflowing_add(value); + + if is_overflow || !is_valid_decimal_precision(new_sum, self.precision) { + // Overflow: set buffer accumulator to null + self.is_not_null.set_bit(group_index, false); + return; + } + + self.sum[group_index] = new_sum; + self.is_not_null.set_bit(group_index, true) + } +} + +fn ensure_bit_capacity(builder: &mut BooleanBufferBuilder, capacity: usize) { + if builder.len() < capacity { + let additional = capacity - builder.len(); + builder.append_n(additional, true); + } +} + +/// Build a boolean buffer from the state and reset the state, based on the emit_to +/// strategy. +fn build_bool_state(state: &mut BooleanBufferBuilder, emit_to: &EmitTo) -> BooleanBuffer { + let bool_state: BooleanBuffer = state.finish(); + + match emit_to { + EmitTo::All => bool_state, + EmitTo::First(n) => { + // split off the first N values in bool_state + let first_n_bools: BooleanBuffer = bool_state.iter().take(*n).collect(); + // reset the existing seen buffer + for seen in bool_state.iter().skip(*n) { + state.append(seen); + } + first_n_bools + } + } +} + +impl GroupsAccumulator for SumDecimalGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> DFResult<()> { + assert!(opt_filter.is_none(), "opt_filter is not supported yet"); + assert_eq!(values.len(), 1); + let values = values[0].as_primitive::(); + let data = values.values(); + + // Update size for the accumulate states + self.sum.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + self.update_single(group_index, value); + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + self.update_single(group_index, value); + } + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> DFResult { + // For each group: + // 1. if `is_empty` is true, it means either there is no value or all values for the group + // are null, in this case we'll return null + // 2. if `is_empty` is false, but `null_state` is true, it means there's an overflow. In + // non-ANSI mode Spark returns null. + let nulls = build_bool_state(&mut self.is_not_null, &emit_to); + let is_empty = build_bool_state(&mut self.is_empty, &emit_to); + let x = (!&is_empty).bitand(&nulls); + + let result = emit_to.take_needed(&mut self.sum); + let result = Decimal128Array::new(result.into(), Some(NullBuffer::new(x))) + .with_data_type(self.result_type.clone()); + + Ok(Arc::new(result)) + } + + fn state(&mut self, emit_to: EmitTo) -> DFResult> { + let nulls = build_bool_state(&mut self.is_not_null, &emit_to); + let nulls = Some(NullBuffer::new(nulls)); + + let sum = emit_to.take_needed(&mut self.sum); + let sum = Decimal128Array::new(sum.into(), nulls.clone()) + .with_data_type(self.result_type.clone()); + + let is_empty = build_bool_state(&mut self.is_empty, &emit_to); + let is_empty = BooleanArray::new(is_empty, None); + + Ok(vec![ + Arc::new(sum) as ArrayRef, + Arc::new(is_empty) as ArrayRef, + ]) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> DFResult<()> { + assert_eq!( + values.len(), + 2, + "Expected two arrays: 'sum' and 'is_empty', but found {}", + values.len() + ); + assert!(opt_filter.is_none(), "opt_filter is not supported yet"); + + // Make sure we have enough capacity for the additional groups + self.sum.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let that_sum = &values[0]; + let that_sum = that_sum.as_primitive::(); + let that_is_empty = &values[1]; + let that_is_empty = that_is_empty + .as_any() + .downcast_ref::() + .unwrap(); + + group_indices + .iter() + .enumerate() + .for_each(|(idx, &group_index)| unsafe { + let this_overflow = self.is_overflow(group_index); + let that_is_empty = that_is_empty.value_unchecked(idx); + let that_overflow = !that_is_empty && that_sum.is_null(idx); + let is_overflow = this_overflow || that_overflow; + + // This part follows the logic in Spark: + // `org.apache.spark.sql.catalyst.expressions.aggregate.Sum` + self.is_not_null.set_bit(group_index, !is_overflow); + self.is_empty.set_bit( + group_index, + self.is_empty.get_bit(group_index) && that_is_empty, + ); + if !is_overflow { + // .. otherwise, the sum value for this particular index must not be null, + // and thus we merge both values and update this sum. + self.sum[group_index] += that_sum.value_unchecked(idx); + } + }); + + Ok(()) + } + + fn size(&self) -> usize { + self.sum.capacity() * std::mem::size_of::() + + self.is_empty.capacity() / 8 + + self.is_not_null.capacity() / 8 + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::datatypes::*; + use arrow_array::builder::{Decimal128Builder, StringBuilder}; + use arrow_array::RecordBatch; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; + use datafusion::physical_plan::memory::MemoryExec; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_common::Result; + use datafusion_expr::AggregateUDF; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr::PhysicalExpr; + use futures::StreamExt; + + #[test] + fn invalid_data_type() { + assert!(SumDecimal::try_new(DataType::Int32).is_err()); + } + + #[tokio::test] + async fn sum_no_overflow() -> Result<()> { + let num_rows = 8192; + let batch = create_record_batch(num_rows); + let mut batches = Vec::new(); + for _ in 0..10 { + batches.push(batch.clone()); + } + let partitions = &[batches]; + let c0: Arc = Arc::new(Column::new("c0", 0)); + let c1: Arc = Arc::new(Column::new("c1", 1)); + + let data_type = DataType::Decimal128(8, 2); + let schema = Arc::clone(&partitions[0][0].schema()); + let scan: Arc = + Arc::new(MemoryExec::try_new(partitions, Arc::clone(&schema), None).unwrap()); + + let aggregate_udf = Arc::new(AggregateUDF::new_from_impl(SumDecimal::try_new( + data_type.clone(), + )?)); + + let aggr_expr = AggregateExprBuilder::new(aggregate_udf, vec![c1]) + .schema(Arc::clone(&schema)) + .alias("sum") + .with_ignore_nulls(false) + .with_distinct(false) + .build()?; + + let aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(vec![(c0, "c0".to_string())]), + vec![aggr_expr.into()], + vec![None], // no filter expressions + scan, + Arc::clone(&schema), + )?); + + let mut stream = aggregate + .execute(0, Arc::new(TaskContext::default())) + .unwrap(); + while let Some(batch) = stream.next().await { + let _batch = batch?; + } + + Ok(()) + } + + fn create_record_batch(num_rows: usize) -> RecordBatch { + let mut decimal_builder = Decimal128Builder::with_capacity(num_rows); + let mut string_builder = StringBuilder::with_capacity(num_rows, num_rows * 32); + for i in 0..num_rows { + decimal_builder.append_value(i as i128); + string_builder.append_value(format!("this is string #{}", i % 1024)); + } + let decimal_array = Arc::new(decimal_builder.finish()); + let string_array = Arc::new(string_builder.finish()); + + let mut fields = vec![]; + let mut columns: Vec = vec![]; + + // string column + fields.push(Field::new("c0", DataType::Utf8, false)); + columns.push(string_array); + + // decimal column + fields.push(Field::new("c1", DataType::Decimal128(38, 10), false)); + columns.push(decimal_array); + + let schema = Schema::new(fields); + RecordBatch::try_new(Arc::new(schema), columns).unwrap() + } +} diff --git a/native/spark-expr/src/agg_funcs/variance.rs b/native/spark-expr/src/agg_funcs/variance.rs new file mode 100644 index 0000000000..e71d713f59 --- /dev/null +++ b/native/spark-expr/src/agg_funcs/variance.rs @@ -0,0 +1,247 @@ +// 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. + +use std::any::Any; + +use arrow::{ + array::{ArrayRef, Float64Array}, + datatypes::{DataType, Field}, +}; +use datafusion::logical_expr::Accumulator; +use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::Volatility::Immutable; +use datafusion_expr::{AggregateUDFImpl, Signature}; +use datafusion_physical_expr::expressions::format_state_name; +use datafusion_physical_expr::expressions::StatsType; + +/// VAR_SAMP and VAR_POP aggregate expression +/// The implementation mostly is the same as the DataFusion's implementation. The reason +/// we have our own implementation is that DataFusion has UInt64 for state_field `count`, +/// while Spark has Double for count. Also we have added `null_on_divide_by_zero` +/// to be consistent with Spark's implementation. +#[derive(Debug)] +pub struct Variance { + name: String, + signature: Signature, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl Variance { + /// Create a new VARIANCE aggregate function + pub fn new( + name: impl Into, + data_type: DataType, + stats_type: StatsType, + null_on_divide_by_zero: bool, + ) -> Self { + // the result of variance just support FLOAT64 data type. + assert!(matches!(data_type, DataType::Float64)); + Self { + name: name.into(), + signature: Signature::numeric(1, Immutable), + stats_type, + null_on_divide_by_zero, + } + } +} + +impl AggregateUDFImpl for Variance { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { + Ok(Box::new(VarianceAccumulator::try_new( + self.stats_type, + self.null_on_divide_by_zero, + )?)) + } + + fn create_sliding_accumulator(&self, _args: AccumulatorArgs) -> Result> { + Ok(Box::new(VarianceAccumulator::try_new( + self.stats_type, + self.null_on_divide_by_zero, + )?)) + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "count"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "mean"), + DataType::Float64, + true, + ), + Field::new(format_state_name(&self.name, "m2"), DataType::Float64, true), + ]) + } + + fn default_value(&self, _data_type: &DataType) -> Result { + Ok(ScalarValue::Float64(None)) + } +} + +/// An accumulator to compute variance +#[derive(Debug)] +pub struct VarianceAccumulator { + m2: f64, + mean: f64, + count: f64, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl VarianceAccumulator { + /// Creates a new `VarianceAccumulator` + pub fn try_new(s_type: StatsType, null_on_divide_by_zero: bool) -> Result { + Ok(Self { + m2: 0_f64, + mean: 0_f64, + count: 0_f64, + stats_type: s_type, + null_on_divide_by_zero, + }) + } + + pub fn get_count(&self) -> f64 { + self.count + } + + pub fn get_mean(&self) -> f64 { + self.mean + } + + pub fn get_m2(&self) -> f64 { + self.m2 + } +} + +impl Accumulator for VarianceAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::from(self.count), + ScalarValue::from(self.mean), + ScalarValue::from(self.m2), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let arr = downcast_value!(&values[0], Float64Array).iter().flatten(); + + for value in arr { + let new_count = self.count + 1.0; + let delta1 = value - self.mean; + let new_mean = delta1 / new_count + self.mean; + let delta2 = value - new_mean; + let new_m2 = self.m2 + delta1 * delta2; + + self.count += 1.0; + self.mean = new_mean; + self.m2 = new_m2; + } + + Ok(()) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let arr = downcast_value!(&values[0], Float64Array).iter().flatten(); + + for value in arr { + let new_count = self.count - 1.0; + let delta1 = self.mean - value; + let new_mean = delta1 / new_count + self.mean; + let delta2 = new_mean - value; + let new_m2 = self.m2 - delta1 * delta2; + + self.count -= 1.0; + self.mean = new_mean; + self.m2 = new_m2; + } + + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let counts = downcast_value!(states[0], Float64Array); + let means = downcast_value!(states[1], Float64Array); + let m2s = downcast_value!(states[2], Float64Array); + + for i in 0..counts.len() { + let c = counts.value(i); + if c == 0_f64 { + continue; + } + let new_count = self.count + c; + let new_mean = self.mean * self.count / new_count + means.value(i) * c / new_count; + let delta = self.mean - means.value(i); + let new_m2 = self.m2 + m2s.value(i) + delta * delta * self.count * c / new_count; + + self.count = new_count; + self.mean = new_mean; + self.m2 = new_m2; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let count = match self.stats_type { + StatsType::Population => self.count, + StatsType::Sample => { + if self.count > 0.0 { + self.count - 1.0 + } else { + self.count + } + } + }; + + Ok(ScalarValue::Float64(match self.count { + count if count == 0.0 => None, + count if count == 1.0 && StatsType::Sample == self.stats_type => { + if self.null_on_divide_by_zero { + None + } else { + Some(f64::NAN) + } + } + _ => Some(self.m2 / count), + })) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 8381dfd2f2..0f59761325 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -154,6 +154,240 @@ impl Hash for Cast { } } +/// Determine if Comet supports a cast, taking options such as EvalMode and Timezone into account. +pub fn cast_supported( + from_type: &DataType, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + + let from_type = if let Dictionary(_, dt) = from_type { + dt + } else { + from_type + }; + + let to_type = if let Dictionary(_, dt) = to_type { + dt + } else { + to_type + }; + + if from_type == to_type { + return true; + } + + match (from_type, to_type) { + (Boolean, _) => can_cast_from_boolean(to_type, options), + (UInt8 | UInt16 | UInt32 | UInt64, Int8 | Int16 | Int32 | Int64) + if options.allow_cast_unsigned_ints => + { + true + } + (Int8, _) => can_cast_from_byte(to_type, options), + (Int16, _) => can_cast_from_short(to_type, options), + (Int32, _) => can_cast_from_int(to_type, options), + (Int64, _) => can_cast_from_long(to_type, options), + (Float32, _) => can_cast_from_float(to_type, options), + (Float64, _) => can_cast_from_double(to_type, options), + (Decimal128(p, s), _) => can_cast_from_decimal(p, s, to_type, options), + (Timestamp(_, None), _) => can_cast_from_timestamp_ntz(to_type, options), + (Timestamp(_, Some(_)), _) => can_cast_from_timestamp(to_type, options), + (Utf8 | LargeUtf8, _) => can_cast_from_string(to_type, options), + (_, Utf8 | LargeUtf8) => can_cast_to_string(from_type, options), + (Struct(from_fields), Struct(to_fields)) => from_fields + .iter() + .zip(to_fields.iter()) + .all(|(a, b)| cast_supported(a.data_type(), b.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_string(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Binary => true, + Float32 | Float64 => { + // https://github.com/apache/datafusion-comet/issues/326 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. + options.allow_incompat + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/325 + // Does not support inputs ending with 'd' or 'f'. Does not support 'inf'. + // Does not support ANSI mode. Returns 0.0 instead of null if input contains no digits + + options.allow_incompat + } + Date32 | Date64 => { + // https://github.com/apache/datafusion-comet/issues/327 + // Only supports years between 262143 BC and 262142 AD + options.allow_incompat + } + Timestamp(_, _) if options.eval_mode == EvalMode::Ansi => { + // ANSI mode not supported + false + } + Timestamp(_, Some(tz)) if tz.as_ref() != "UTC" => { + // Cast will use UTC instead of $timeZoneId + options.allow_incompat + } + Timestamp(_, _) => { + // https://github.com/apache/datafusion-comet/issues/328 + // Not all valid formats are supported + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_to_string(from_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match from_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Date32 | Date64 | Timestamp(_, _) => true, + Float32 | Float64 => { + // There can be differences in precision. + // For example, the input \"1.4E-45\" will produce 1.0E-45 " + + // instead of 1.4E-45")) + true + } + Decimal128(_, _) => { + // https://github.com/apache/datafusion-comet/issues/1068 + // There can be formatting differences in some case due to Spark using + // scientific notation where Comet does not + true + } + Binary => { + // https://github.com/apache/datafusion-comet/issues/377 + // Only works for binary data representing valid UTF-8 strings + options.allow_incompat + } + Struct(fields) => fields + .iter() + .all(|f| can_cast_to_string(f.data_type(), options)), + _ => false, + } +} + +fn can_cast_from_timestamp_ntz(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Timestamp(_, _) | Date32 | Date64 | Utf8 => { + // incompatible + options.allow_incompat + } + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_timestamp(to_type: &DataType, _options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 => { + // https://github.com/apache/datafusion-comet/issues/352 + // this seems like an edge case that isn't important for us to support + false + } + Int64 => { + // https://github.com/apache/datafusion-comet/issues/352 + true + } + Date32 | Date64 | Utf8 | Decimal128(_, _) => true, + _ => { + // unsupported + false + } + } +} + +fn can_cast_from_boolean(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!(to_type, Int8 | Int16 | Int32 | Int64 | Float32 | Float64) +} + +fn can_cast_from_byte(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_short(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_int(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Utf8 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_long(to_type: &DataType, options: &SparkCastOptions) -> bool { + use DataType::*; + match to_type { + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(_, _) => { + // incompatible: no overflow check + options.allow_incompat + } + _ => false, + } +} + +fn can_cast_from_float(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float64 | Decimal128(_, _) + ) +} + +fn can_cast_from_double(to_type: &DataType, _: &SparkCastOptions) -> bool { + use DataType::*; + matches!( + to_type, + Boolean | Int8 | Int16 | Int32 | Int64 | Float32 | Decimal128(_, _) + ) +} + +fn can_cast_from_decimal( + p1: &u8, + _s1: &i8, + to_type: &DataType, + options: &SparkCastOptions, +) -> bool { + use DataType::*; + match to_type { + Int8 | Int16 | Int32 | Int64 | Float32 | Float64 => true, + Decimal128(p2, _) => { + if p2 < p1 { + // https://github.com/apache/datafusion/issues/13492 + // Incompatible(Some("Casting to smaller precision is not supported")) + options.allow_incompat + } else { + true + } + } + _ => false, + } +} + macro_rules! cast_utf8_to_int { ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{ let len = $array.len(); diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index 71ff0e9dcc..27c77d7f2b 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -15,14 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::scalar_funcs::hash_expressions::{ - spark_sha224, spark_sha256, spark_sha384, spark_sha512, -}; +use crate::hash_funcs::*; use crate::scalar_funcs::{ - spark_ceil, spark_date_add, spark_date_sub, spark_decimal_div, spark_floor, spark_hex, - spark_isnan, spark_make_decimal, spark_murmur3_hash, spark_read_side_padding, spark_round, - spark_unhex, spark_unscaled_value, spark_xxhash64, SparkChrFunc, + spark_ceil, spark_decimal_div, spark_floor, spark_hex, spark_isnan, spark_make_decimal, + spark_round, spark_unhex, spark_unscaled_value, SparkChrFunc, }; +use crate::{spark_date_add, spark_date_sub, spark_read_side_padding}; use arrow_schema::DataType; use datafusion_common::{DataFusionError, Result as DataFusionResult}; use datafusion_expr::registry::FunctionRegistry; diff --git a/native/spark-expr/src/datetime_funcs/date_arithmetic.rs b/native/spark-expr/src/datetime_funcs/date_arithmetic.rs new file mode 100644 index 0000000000..cc4da9af70 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/date_arithmetic.rs @@ -0,0 +1,102 @@ +// 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. + +use arrow::array::{ArrayRef, AsArray}; +use arrow::compute::kernels::numeric::{add, sub}; +use arrow::datatypes::IntervalDayTime; +use arrow_array::builder::IntervalDayTimeBuilder; +use arrow_array::types::{Int16Type, Int32Type, Int8Type}; +use arrow_array::{Array, Datum}; +use arrow_schema::{ArrowError, DataType}; +use datafusion::physical_expr_common::datum; +use datafusion::physical_plan::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue}; +use std::sync::Arc; + +macro_rules! scalar_date_arithmetic { + ($start:expr, $days:expr, $op:expr) => {{ + let interval = IntervalDayTime::new(*$days as i32, 0); + let interval_cv = ColumnarValue::Scalar(ScalarValue::IntervalDayTime(Some(interval))); + datum::apply($start, &interval_cv, $op) + }}; +} +macro_rules! array_date_arithmetic { + ($days:expr, $interval_builder:expr, $intType:ty) => {{ + for day in $days.as_primitive::<$intType>().into_iter() { + if let Some(non_null_day) = day { + $interval_builder.append_value(IntervalDayTime::new(non_null_day as i32, 0)); + } else { + $interval_builder.append_null(); + } + } + }}; +} + +/// Spark-compatible `date_add` and `date_sub` expressions, which assumes days for the second +/// argument, but we cannot directly add that to a Date32. We generate an IntervalDayTime from the +/// second argument and use DataFusion's interface to apply Arrow's operators. +fn spark_date_arithmetic( + args: &[ColumnarValue], + op: impl Fn(&dyn Datum, &dyn Datum) -> Result, +) -> Result { + let start = &args[0]; + match &args[1] { + ColumnarValue::Scalar(ScalarValue::Int8(Some(days))) => { + scalar_date_arithmetic!(start, days, op) + } + ColumnarValue::Scalar(ScalarValue::Int16(Some(days))) => { + scalar_date_arithmetic!(start, days, op) + } + ColumnarValue::Scalar(ScalarValue::Int32(Some(days))) => { + scalar_date_arithmetic!(start, days, op) + } + ColumnarValue::Array(days) => { + let mut interval_builder = IntervalDayTimeBuilder::with_capacity(days.len()); + match days.data_type() { + DataType::Int8 => { + array_date_arithmetic!(days, interval_builder, Int8Type) + } + DataType::Int16 => { + array_date_arithmetic!(days, interval_builder, Int16Type) + } + DataType::Int32 => { + array_date_arithmetic!(days, interval_builder, Int32Type) + } + _ => { + return Err(DataFusionError::Internal(format!( + "Unsupported data types {:?} for date arithmetic.", + args, + ))) + } + } + let interval_cv = ColumnarValue::Array(Arc::new(interval_builder.finish())); + datum::apply(start, &interval_cv, op) + } + _ => Err(DataFusionError::Internal(format!( + "Unsupported data types {:?} for date arithmetic.", + args, + ))), + } +} + +pub fn spark_date_add(args: &[ColumnarValue]) -> Result { + spark_date_arithmetic(args, add) +} + +pub fn spark_date_sub(args: &[ColumnarValue]) -> Result { + spark_date_arithmetic(args, sub) +} diff --git a/native/spark-expr/src/datetime_funcs/date_trunc.rs b/native/spark-expr/src/datetime_funcs/date_trunc.rs new file mode 100644 index 0000000000..5c044945d0 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/date_trunc.rs @@ -0,0 +1,113 @@ +// 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. + +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue::Utf8}; +use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; + +use crate::kernels::temporal::{date_trunc_array_fmt_dyn, date_trunc_dyn}; + +#[derive(Debug, Eq)] +pub struct DateTruncExpr { + /// An array with DataType::Date32 + child: Arc, + /// Scalar UTF8 string matching the valid values in Spark SQL: https://spark.apache.org/docs/latest/api/sql/index.html#trunc + format: Arc, +} + +impl Hash for DateTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + } +} +impl PartialEq for DateTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.format.eq(&other.format) + } +} + +impl DateTruncExpr { + pub fn new(child: Arc, format: Arc) -> Self { + DateTruncExpr { child, format } + } +} + +impl Display for DateTruncExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "DateTrunc [child:{}, format: {}]", + self.child, self.format + ) + } +} + +impl PhysicalExpr for DateTruncExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + self.child.data_type(input_schema) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let date = self.child.evaluate(batch)?; + let format = self.format.evaluate(batch)?; + match (date, format) { + (ColumnarValue::Array(date), ColumnarValue::Scalar(Utf8(Some(format)))) => { + let result = date_trunc_dyn(&date, format)?; + Ok(ColumnarValue::Array(result)) + } + (ColumnarValue::Array(date), ColumnarValue::Array(formats)) => { + let result = date_trunc_array_fmt_dyn(&date, &formats)?; + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Invalid input to function DateTrunc. Expected (PrimitiveArray, Scalar) or \ + (PrimitiveArray, StringArray)".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result, DataFusionError> { + Ok(Arc::new(DateTruncExpr::new( + Arc::clone(&children[0]), + Arc::clone(&self.format), + ))) + } +} diff --git a/native/spark-expr/src/datetime_funcs/hour.rs b/native/spark-expr/src/datetime_funcs/hour.rs new file mode 100644 index 0000000000..faf9529a51 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/hour.rs @@ -0,0 +1,122 @@ +// 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. + +use crate::utils::array_with_timezone; +use arrow::{ + compute::{date_part, DatePart}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct HourExpr { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc, + timezone: String, +} + +impl Hash for HourExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for HourExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + +impl HourExpr { + pub fn new(child: Arc, timezone: String) -> Self { + HourExpr { child, timezone } + } +} + +impl Display for HourExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Hour [timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PhysicalExpr for HourExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + )?; + let result = date_part(&array, DatePart::Hour)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Hour(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result, DataFusionError> { + Ok(Arc::new(HourExpr::new( + Arc::clone(&children[0]), + self.timezone.clone(), + ))) + } +} diff --git a/native/spark-expr/src/datetime_funcs/minute.rs b/native/spark-expr/src/datetime_funcs/minute.rs new file mode 100644 index 0000000000..b7facc1673 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/minute.rs @@ -0,0 +1,122 @@ +// 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. + +use crate::utils::array_with_timezone; +use arrow::{ + compute::{date_part, DatePart}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct MinuteExpr { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc, + timezone: String, +} + +impl Hash for MinuteExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for MinuteExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + +impl MinuteExpr { + pub fn new(child: Arc, timezone: String) -> Self { + MinuteExpr { child, timezone } + } +} + +impl Display for MinuteExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Minute [timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PhysicalExpr for MinuteExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + )?; + let result = date_part(&array, DatePart::Minute)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Minute(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result, DataFusionError> { + Ok(Arc::new(MinuteExpr::new( + Arc::clone(&children[0]), + self.timezone.clone(), + ))) + } +} diff --git a/native/spark-expr/src/datetime_funcs/mod.rs b/native/spark-expr/src/datetime_funcs/mod.rs new file mode 100644 index 0000000000..1f4d427282 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/mod.rs @@ -0,0 +1,30 @@ +// 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. + +mod date_arithmetic; +mod date_trunc; +mod hour; +mod minute; +mod second; +mod timestamp_trunc; + +pub use date_arithmetic::{spark_date_add, spark_date_sub}; +pub use date_trunc::DateTruncExpr; +pub use hour::HourExpr; +pub use minute::MinuteExpr; +pub use second::SecondExpr; +pub use timestamp_trunc::TimestampTruncExpr; diff --git a/native/spark-expr/src/datetime_funcs/second.rs b/native/spark-expr/src/datetime_funcs/second.rs new file mode 100644 index 0000000000..76a4dd9a2c --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/second.rs @@ -0,0 +1,122 @@ +// 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. + +use crate::utils::array_with_timezone; +use arrow::{ + compute::{date_part, DatePart}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct SecondExpr { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc, + timezone: String, +} + +impl Hash for SecondExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for SecondExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) + } +} + +impl SecondExpr { + pub fn new(child: Arc, timezone: String) -> Self { + SecondExpr { child, timezone } + } +} + +impl Display for SecondExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Second (timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PhysicalExpr for SecondExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + )?; + let result = date_part(&array, DatePart::Second)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Second(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result, DataFusionError> { + Ok(Arc::new(SecondExpr::new( + Arc::clone(&children[0]), + self.timezone.clone(), + ))) + } +} diff --git a/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs b/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs new file mode 100644 index 0000000000..349992322f --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs @@ -0,0 +1,152 @@ +// 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. + +use crate::utils::array_with_timezone; +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue::Utf8}; +use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hash; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; + +use crate::kernels::temporal::{timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn}; + +#[derive(Debug, Eq)] +pub struct TimestampTruncExpr { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc, + /// Scalar UTF8 string matching the valid values in Spark SQL: https://spark.apache.org/docs/latest/api/sql/index.html#date_trunc + format: Arc, + /// String containing a timezone name. The name must be found in the standard timezone + /// database (https://en.wikipedia.org/wiki/List_of_tz_database_time_zones). The string is + /// later parsed into a chrono::TimeZone. + /// Timestamp arrays in this implementation are kept in arrays of UTC timestamps (in micros) + /// along with a single value for the associated TimeZone. The timezone offset is applied + /// just before any operations on the timestamp + timezone: String, +} + +impl Hash for TimestampTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + self.timezone.hash(state); + } +} +impl PartialEq for TimestampTruncExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.format.eq(&other.format) + && self.timezone.eq(&other.timezone) + } +} + +impl TimestampTruncExpr { + pub fn new( + child: Arc, + format: Arc, + timezone: String, + ) -> Self { + TimestampTruncExpr { + child, + format, + timezone, + } + } +} + +impl Display for TimestampTruncExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "TimestampTrunc [child:{}, format:{}, timezone: {}]", + self.child, self.format, self.timezone + ) + } +} + +impl PhysicalExpr for TimestampTruncExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + match self.child.data_type(input_schema)? { + DataType::Dictionary(key_type, _) => Ok(DataType::Dictionary( + key_type, + Box::new(DataType::Timestamp(Microsecond, None)), + )), + _ => Ok(DataType::Timestamp(Microsecond, None)), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let timestamp = self.child.evaluate(batch)?; + let format = self.format.evaluate(batch)?; + let tz = self.timezone.clone(); + match (timestamp, format) { + (ColumnarValue::Array(ts), ColumnarValue::Scalar(Utf8(Some(format)))) => { + let ts = array_with_timezone( + ts, + tz.clone(), + Some(&DataType::Timestamp(Microsecond, Some(tz.into()))), + )?; + let result = timestamp_trunc_dyn(&ts, format)?; + Ok(ColumnarValue::Array(result)) + } + (ColumnarValue::Array(ts), ColumnarValue::Array(formats)) => { + let ts = array_with_timezone( + ts, + tz.clone(), + Some(&DataType::Timestamp(Microsecond, Some(tz.into()))), + )?; + let result = timestamp_trunc_array_fmt_dyn(&ts, &formats)?; + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Invalid input to function TimestampTrunc. \ + Expected (PrimitiveArray, Scalar, String) or \ + (PrimitiveArray, StringArray, String)" + .to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result, DataFusionError> { + Ok(Arc::new(TimestampTruncExpr::new( + Arc::clone(&children[0]), + Arc::clone(&self.format), + self.timezone.clone(), + ))) + } +} diff --git a/native/spark-expr/src/hash_funcs/mod.rs b/native/spark-expr/src/hash_funcs/mod.rs new file mode 100644 index 0000000000..7649c4c547 --- /dev/null +++ b/native/spark-expr/src/hash_funcs/mod.rs @@ -0,0 +1,25 @@ +// 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. + +pub mod murmur3; +mod sha2; +pub(super) mod utils; +mod xxhash64; + +pub use murmur3::spark_murmur3_hash; +pub use sha2::{spark_sha224, spark_sha256, spark_sha384, spark_sha512}; +pub use xxhash64::spark_xxhash64; diff --git a/native/spark-expr/src/hash_funcs/murmur3.rs b/native/spark-expr/src/hash_funcs/murmur3.rs new file mode 100644 index 0000000000..3ed70ba741 --- /dev/null +++ b/native/spark-expr/src/hash_funcs/murmur3.rs @@ -0,0 +1,280 @@ +// 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. + +use crate::create_hashes_internal; +use arrow::compute::take; +use arrow_array::types::ArrowDictionaryKeyType; +use arrow_array::{Array, ArrayRef, ArrowNativeTypeOp, DictionaryArray, Int32Array}; +use arrow_buffer::ArrowNativeType; +use datafusion_common::{internal_err, DataFusionError, ScalarValue}; +use datafusion_expr::ColumnarValue; +use std::sync::Arc; + +/// Spark compatible murmur3 hash (just `hash` in Spark) in vectorized execution fashion +pub fn spark_murmur3_hash(args: &[ColumnarValue]) -> Result { + let length = args.len(); + let seed = &args[length - 1]; + match seed { + ColumnarValue::Scalar(ScalarValue::Int32(Some(seed))) => { + // iterate over the arguments to find out the length of the array + let num_rows = args[0..args.len() - 1] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + let mut hashes: Vec = vec![0_u32; num_rows]; + hashes.fill(*seed as u32); + let arrays = args[0..args.len() - 1] + .iter() + .map(|arg| match arg { + ColumnarValue::Array(array) => Arc::clone(array), + ColumnarValue::Scalar(scalar) => { + scalar.clone().to_array_of_size(num_rows).unwrap() + } + }) + .collect::>(); + create_murmur3_hashes(&arrays, &mut hashes)?; + if num_rows == 1 { + Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some( + hashes[0] as i32, + )))) + } else { + let hashes: Vec = hashes.into_iter().map(|x| x as i32).collect(); + Ok(ColumnarValue::Array(Arc::new(Int32Array::from(hashes)))) + } + } + _ => { + internal_err!( + "The seed of function murmur3_hash must be an Int32 scalar value, but got: {:?}.", + seed + ) + } + } +} + +/// Spark-compatible murmur3 hash function +#[inline] +pub fn spark_compatible_murmur3_hash>(data: T, seed: u32) -> u32 { + #[inline] + fn mix_k1(mut k1: i32) -> i32 { + k1 = k1.mul_wrapping(0xcc9e2d51u32 as i32); + k1 = k1.rotate_left(15); + k1 = k1.mul_wrapping(0x1b873593u32 as i32); + k1 + } + + #[inline] + fn mix_h1(mut h1: i32, k1: i32) -> i32 { + h1 ^= k1; + h1 = h1.rotate_left(13); + h1 = h1.mul_wrapping(5).add_wrapping(0xe6546b64u32 as i32); + h1 + } + + #[inline] + fn fmix(mut h1: i32, len: i32) -> i32 { + h1 ^= len; + h1 ^= (h1 as u32 >> 16) as i32; + h1 = h1.mul_wrapping(0x85ebca6bu32 as i32); + h1 ^= (h1 as u32 >> 13) as i32; + h1 = h1.mul_wrapping(0xc2b2ae35u32 as i32); + h1 ^= (h1 as u32 >> 16) as i32; + h1 + } + + #[inline] + unsafe fn hash_bytes_by_int(data: &[u8], seed: u32) -> i32 { + // safety: data length must be aligned to 4 bytes + let mut h1 = seed as i32; + for i in (0..data.len()).step_by(4) { + let ints = data.as_ptr().add(i) as *const i32; + let mut half_word = ints.read_unaligned(); + if cfg!(target_endian = "big") { + half_word = half_word.reverse_bits(); + } + h1 = mix_h1(h1, mix_k1(half_word)); + } + h1 + } + let data = data.as_ref(); + let len = data.len(); + let len_aligned = len - len % 4; + + // safety: + // avoid boundary checking in performance critical codes. + // all operations are guaranteed to be safe + // data is &[u8] so we do not need to check for proper alignment + unsafe { + let mut h1 = if len_aligned > 0 { + hash_bytes_by_int(&data[0..len_aligned], seed) + } else { + seed as i32 + }; + + for i in len_aligned..len { + let half_word = *data.get_unchecked(i) as i8 as i32; + h1 = mix_h1(h1, mix_k1(half_word)); + } + fmix(h1, len as i32) as u32 + } +} + +/// Hash the values in a dictionary array +fn create_hashes_dictionary( + array: &ArrayRef, + hashes_buffer: &mut [u32], + first_col: bool, +) -> datafusion_common::Result<()> { + let dict_array = array.as_any().downcast_ref::>().unwrap(); + if !first_col { + // unpack the dictionary array as each row may have a different hash input + let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; + create_murmur3_hashes(&[unpacked], hashes_buffer)?; + } else { + // For the first column, hash each dictionary value once, and then use + // that computed hash for each key value to avoid a potentially + // expensive redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + // same initial seed as Spark + let mut dict_hashes = vec![42; dict_values.len()]; + create_murmur3_hashes(&[dict_values], &mut dict_hashes)?; + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } + } + Ok(()) +} + +/// Creates hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +pub fn create_murmur3_hashes<'a>( + arrays: &[ArrayRef], + hashes_buffer: &'a mut [u32], +) -> datafusion_common::Result<&'a mut [u32]> { + create_hashes_internal!( + arrays, + hashes_buffer, + spark_compatible_murmur3_hash, + create_hashes_dictionary + ); + Ok(hashes_buffer) +} + +#[cfg(test)] +mod tests { + use arrow::array::{Float32Array, Float64Array}; + use std::sync::Arc; + + use crate::murmur3::create_murmur3_hashes; + use crate::test_hashes_with_nulls; + use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; + + fn test_murmur3_hash>> + 'static>( + values: Vec>, + expected: Vec, + ) { + test_hashes_with_nulls!(create_murmur3_hashes, T, values, expected, u32); + } + + #[test] + fn test_i8() { + test_murmur3_hash::( + vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365], + ); + } + + #[test] + fn test_i32() { + test_murmur3_hash::( + vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6], + ); + } + + #[test] + fn test_i64() { + test_murmur3_hash::( + vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], + vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb], + ); + } + + #[test] + fn test_f32() { + test_murmur3_hash::( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, + ], + ); + } + + #[test] + fn test_f64() { + test_murmur3_hash::( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, + ], + ); + } + + #[test] + fn test_str() { + let input = [ + "hello", "bar", "", "😁", "天地", "a", "ab", "abc", "abcd", "abcde", + ] + .iter() + .map(|s| Some(s.to_string())) + .collect::>>(); + let expected: Vec = vec![ + 3286402344, 2486176763, 142593372, 885025535, 2395000894, 1485273170, 0xfa37157b, + 1322437556, 0xe860e5cc, 814637928, + ]; + + test_murmur3_hash::(input.clone(), expected); + } +} diff --git a/native/spark-expr/src/hash_funcs/sha2.rs b/native/spark-expr/src/hash_funcs/sha2.rs new file mode 100644 index 0000000000..90917a9eb4 --- /dev/null +++ b/native/spark-expr/src/hash_funcs/sha2.rs @@ -0,0 +1,76 @@ +// 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. + +use crate::scalar_funcs::hex_strings; +use arrow_array::{Array, StringArray}; +use datafusion::functions::crypto::{sha224, sha256, sha384, sha512}; +use datafusion_common::cast::as_binary_array; +use datafusion_common::{exec_err, DataFusionError, ScalarValue}; +use datafusion_expr::{ColumnarValue, ScalarUDF}; +use std::sync::Arc; + +/// `sha224` function that simulates Spark's `sha2` expression with bit width 224 +pub fn spark_sha224(args: &[ColumnarValue]) -> Result { + wrap_digest_result_as_hex_string(args, sha224()) +} + +/// `sha256` function that simulates Spark's `sha2` expression with bit width 0 or 256 +pub fn spark_sha256(args: &[ColumnarValue]) -> Result { + wrap_digest_result_as_hex_string(args, sha256()) +} + +/// `sha384` function that simulates Spark's `sha2` expression with bit width 384 +pub fn spark_sha384(args: &[ColumnarValue]) -> Result { + wrap_digest_result_as_hex_string(args, sha384()) +} + +/// `sha512` function that simulates Spark's `sha2` expression with bit width 512 +pub fn spark_sha512(args: &[ColumnarValue]) -> Result { + wrap_digest_result_as_hex_string(args, sha512()) +} + +// Spark requires hex string as the result of sha2 functions, we have to wrap the +// result of digest functions as hex string +fn wrap_digest_result_as_hex_string( + args: &[ColumnarValue], + digest: Arc, +) -> Result { + let row_count = match &args[0] { + ColumnarValue::Array(array) => array.len(), + ColumnarValue::Scalar(_) => 1, + }; + let value = digest.invoke_batch(args, row_count)?; + match value { + ColumnarValue::Array(array) => { + let binary_array = as_binary_array(&array)?; + let string_array: StringArray = binary_array + .iter() + .map(|opt| opt.map(hex_strings::<_>)) + .collect(); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + ColumnarValue::Scalar(ScalarValue::Binary(opt)) => Ok(ColumnarValue::Scalar( + ScalarValue::Utf8(opt.map(hex_strings::<_>)), + )), + _ => { + exec_err!( + "digest function should return binary value, but got: {:?}", + value.data_type() + ) + } + } +} diff --git a/native/spark-expr/src/hash_funcs/utils.rs b/native/spark-expr/src/hash_funcs/utils.rs new file mode 100644 index 0000000000..07ba1952d7 --- /dev/null +++ b/native/spark-expr/src/hash_funcs/utils.rs @@ -0,0 +1,393 @@ +// 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. + +//! This includes utilities for hashing and murmur3 hashing. + +#[macro_export] +macro_rules! hash_array { + ($array_type: ident, $column: ident, $hashes: ident, $hash_method: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = $hash_method(&array.value(i), *hash); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = $hash_method(&array.value(i), *hash); + } + } + } + }; +} + +#[macro_export] +macro_rules! hash_array_boolean { + ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident, $hash_method: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = + $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); + } + } + } + }; +} + +#[macro_export] +macro_rules! hash_array_primitive { + ($array_type: ident, $column: ident, $ty: ident, $hashes: ident, $hash_method: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let values = array.values(); + + if array.null_count() == 0 { + for (hash, value) in $hashes.iter_mut().zip(values.iter()) { + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); + } + } else { + for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { + if !array.is_null(i) { + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); + } + } + } + }; +} + +#[macro_export] +macro_rules! hash_array_primitive_float { + ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident, $hash_method: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let values = array.values(); + + if array.null_count() == 0 { + for (hash, value) in $hashes.iter_mut().zip(values.iter()) { + // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. + if *value == 0.0 && value.is_sign_negative() { + *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); + } else { + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); + } + } + } else { + for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { + if !array.is_null(i) { + // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. + if *value == 0.0 && value.is_sign_negative() { + *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); + } else { + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); + } + } + } + } + }; +} + +#[macro_export] +macro_rules! hash_array_decimal { + ($array_type:ident, $column: ident, $hashes: ident, $hash_method: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = $hash_method(array.value(i).to_le_bytes(), *hash); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = $hash_method(array.value(i).to_le_bytes(), *hash); + } + } + } + }; +} + +/// Creates hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +/// +/// `hash_method` is the hash function to use. +/// `create_dictionary_hash_method` is the function to create hashes for dictionary arrays input. +#[macro_export] +macro_rules! create_hashes_internal { + ($arrays: ident, $hashes_buffer: ident, $hash_method: ident, $create_dictionary_hash_method: ident) => { + use arrow::datatypes::{DataType, TimeUnit}; + use arrow_array::{types::*, *}; + + for (i, col) in $arrays.iter().enumerate() { + let first_col = i == 0; + match col.data_type() { + DataType::Boolean => { + $crate::hash_array_boolean!( + BooleanArray, + col, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Int8 => { + $crate::hash_array_primitive!( + Int8Array, + col, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Int16 => { + $crate::hash_array_primitive!( + Int16Array, + col, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Int32 => { + $crate::hash_array_primitive!( + Int32Array, + col, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Int64 => { + $crate::hash_array_primitive!( + Int64Array, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Float32 => { + $crate::hash_array_primitive_float!( + Float32Array, + col, + f32, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Float64 => { + $crate::hash_array_primitive_float!( + Float64Array, + col, + f64, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Second, _) => { + $crate::hash_array_primitive!( + TimestampSecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Millisecond, _) => { + $crate::hash_array_primitive!( + TimestampMillisecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + $crate::hash_array_primitive!( + TimestampMicrosecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Nanosecond, _) => { + $crate::hash_array_primitive!( + TimestampNanosecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Date32 => { + $crate::hash_array_primitive!( + Date32Array, + col, + i32, + $hashes_buffer, + $hash_method + ); + } + DataType::Date64 => { + $crate::hash_array_primitive!( + Date64Array, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Utf8 => { + $crate::hash_array!(StringArray, col, $hashes_buffer, $hash_method); + } + DataType::LargeUtf8 => { + $crate::hash_array!(LargeStringArray, col, $hashes_buffer, $hash_method); + } + DataType::Binary => { + $crate::hash_array!(BinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::LargeBinary => { + $crate::hash_array!(LargeBinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::FixedSizeBinary(_) => { + $crate::hash_array!(FixedSizeBinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::Decimal128(_, _) => { + $crate::hash_array_decimal!(Decimal128Array, col, $hashes_buffer, $hash_method); + } + DataType::Dictionary(index_type, _) => match **index_type { + DataType::Int8 => { + $create_dictionary_hash_method::(col, $hashes_buffer, first_col)?; + } + DataType::Int16 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::Int32 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::Int64 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt8 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt16 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt32 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt64 => { + $create_dictionary_hash_method::( + col, + $hashes_buffer, + first_col, + )?; + } + _ => { + return Err(DataFusionError::Internal(format!( + "Unsupported dictionary type in hasher hashing: {}", + col.data_type(), + ))) + } + }, + _ => { + // This is internal because we should have caught this before. + return Err(DataFusionError::Internal(format!( + "Unsupported data type in hasher: {}", + col.data_type() + ))); + } + } + } + }; +} + +pub(crate) mod test_utils { + + #[macro_export] + macro_rules! test_hashes_internal { + ($hash_method: ident, $input: expr, $initial_seeds: expr, $expected: expr) => { + let i = $input; + let mut hashes = $initial_seeds.clone(); + $hash_method(&[i], &mut hashes).unwrap(); + assert_eq!(hashes, $expected); + }; + } + + #[macro_export] + macro_rules! test_hashes_with_nulls { + ($method: ident, $t: ty, $values: ident, $expected: ident, $seed_type: ty) => { + // copied before inserting nulls + let mut input_with_nulls = $values.clone(); + let mut expected_with_nulls = $expected.clone(); + // test before inserting nulls + let len = $values.len(); + let initial_seeds = vec![42 as $seed_type; len]; + let i = Arc::new(<$t>::from($values)) as ArrayRef; + $crate::test_hashes_internal!($method, i, initial_seeds, $expected); + + // test with nulls + let median = len / 2; + input_with_nulls.insert(0, None); + input_with_nulls.insert(median, None); + expected_with_nulls.insert(0, 42 as $seed_type); + expected_with_nulls.insert(median, 42 as $seed_type); + let len_with_nulls = len + 2; + let initial_seeds_with_nulls = vec![42 as $seed_type; len_with_nulls]; + let nullable_input = Arc::new(<$t>::from(input_with_nulls)) as ArrayRef; + $crate::test_hashes_internal!( + $method, + nullable_input, + initial_seeds_with_nulls, + expected_with_nulls + ); + }; + } +} diff --git a/native/spark-expr/src/hash_funcs/xxhash64.rs b/native/spark-expr/src/hash_funcs/xxhash64.rs new file mode 100644 index 0000000000..e96f178d83 --- /dev/null +++ b/native/spark-expr/src/hash_funcs/xxhash64.rs @@ -0,0 +1,264 @@ +// 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. + +use arrow::compute::take; +use twox_hash::XxHash64; + +use datafusion::{ + arrow::{ + array::*, + datatypes::{ArrowDictionaryKeyType, ArrowNativeType}, + }, + common::{internal_err, ScalarValue}, + error::{DataFusionError, Result}, +}; + +use crate::create_hashes_internal; +use arrow_array::{Array, ArrayRef, Int64Array}; +use datafusion_expr::ColumnarValue; +use std::sync::Arc; + +/// Spark compatible xxhash64 in vectorized execution fashion +pub fn spark_xxhash64(args: &[ColumnarValue]) -> Result { + let length = args.len(); + let seed = &args[length - 1]; + match seed { + ColumnarValue::Scalar(ScalarValue::Int64(Some(seed))) => { + // iterate over the arguments to find out the length of the array + let num_rows = args[0..args.len() - 1] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + let mut hashes: Vec = vec![0_u64; num_rows]; + hashes.fill(*seed as u64); + let arrays = args[0..args.len() - 1] + .iter() + .map(|arg| match arg { + ColumnarValue::Array(array) => Arc::clone(array), + ColumnarValue::Scalar(scalar) => { + scalar.clone().to_array_of_size(num_rows).unwrap() + } + }) + .collect::>(); + create_xxhash64_hashes(&arrays, &mut hashes)?; + if num_rows == 1 { + Ok(ColumnarValue::Scalar(ScalarValue::Int64(Some( + hashes[0] as i64, + )))) + } else { + let hashes: Vec = hashes.into_iter().map(|x| x as i64).collect(); + Ok(ColumnarValue::Array(Arc::new(Int64Array::from(hashes)))) + } + } + _ => { + internal_err!( + "The seed of function xxhash64 must be an Int64 scalar value, but got: {:?}.", + seed + ) + } + } +} + +#[inline] +fn spark_compatible_xxhash64>(data: T, seed: u64) -> u64 { + XxHash64::oneshot(seed, data.as_ref()) +} + +// Hash the values in a dictionary array using xxhash64 +fn create_xxhash64_hashes_dictionary( + array: &ArrayRef, + hashes_buffer: &mut [u64], + first_col: bool, +) -> Result<()> { + let dict_array = array.as_any().downcast_ref::>().unwrap(); + if !first_col { + let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; + create_xxhash64_hashes(&[unpacked], hashes_buffer)?; + } else { + // Hash each dictionary value once, and then use that computed + // hash for each key value to avoid a potentially expensive + // redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + // same initial seed as Spark + let mut dict_hashes = vec![42u64; dict_values.len()]; + create_xxhash64_hashes(&[dict_values], &mut dict_hashes)?; + + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } + } + Ok(()) +} + +/// Creates xxhash64 hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +fn create_xxhash64_hashes<'a>( + arrays: &[ArrayRef], + hashes_buffer: &'a mut [u64], +) -> Result<&'a mut [u64]> { + create_hashes_internal!( + arrays, + hashes_buffer, + spark_compatible_xxhash64, + create_xxhash64_hashes_dictionary + ); + Ok(hashes_buffer) +} + +#[cfg(test)] +mod tests { + use arrow::array::{Float32Array, Float64Array}; + use std::sync::Arc; + + use super::create_xxhash64_hashes; + use crate::test_hashes_with_nulls; + use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; + + fn test_xxhash64_hash>> + 'static>( + values: Vec>, + expected: Vec, + ) { + test_hashes_with_nulls!(create_xxhash64_hashes, T, values, expected, u64); + } + + #[test] + fn test_i8() { + test_xxhash64_hash::( + vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], + vec![ + 0xa309b38455455929, + 0x3229fbc4681e48f3, + 0x1bfdda8861c06e45, + 0x77cc15d9f9f2cdc2, + 0x39bc22b9e94d81d0, + ], + ); + } + + #[test] + fn test_i32() { + test_xxhash64_hash::( + vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], + vec![ + 0xa309b38455455929, + 0x3229fbc4681e48f3, + 0x1bfdda8861c06e45, + 0x14f0ac009c21721c, + 0x1cc7cb8d034769cd, + ], + ); + } + + #[test] + fn test_i64() { + test_xxhash64_hash::( + vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], + vec![ + 0x9ed50fd59358d232, + 0xb71b47ebda15746c, + 0x358ae035bfb46fd2, + 0xd2f1c616ae7eb306, + 0x88608019c494c1f4, + ], + ); + } + + #[test] + fn test_f32() { + test_xxhash64_hash::( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0x9b92689757fcdbd, + 0x3229fbc4681e48f3, + 0x3229fbc4681e48f3, + 0xa2becc0e61bb3823, + 0x8f20ab82d4f3687f, + 0xdce4982d97f7ac4, + ], + ) + } + + #[test] + fn test_f64() { + test_xxhash64_hash::( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0xe1fd6e07fee8ad53, + 0xb71b47ebda15746c, + 0xb71b47ebda15746c, + 0x8cdde022746f8f1f, + 0x793c5c88d313eac7, + 0xc5e60e7b75d9b232, + ], + ) + } + + #[test] + fn test_str() { + let input = [ + "hello", "bar", "", "😁", "天地", "a", "ab", "abc", "abcd", "abcde", + ] + .iter() + .map(|s| Some(s.to_string())) + .collect::>>(); + + test_xxhash64_hash::( + input, + vec![ + 0xc3629e6318d53932, + 0xe7097b6a54378d8a, + 0x98b1582b0977e704, + 0xa80d9d5a6a523bd5, + 0xfcba5f61ac666c61, + 0x88e4fe59adf7b0cc, + 0x259dd873209a3fe3, + 0x13c1d910702770e6, + 0xa17b5eb5dc364dff, + 0xf241303e4a90f299, + ], + ) + } +} diff --git a/native/spark-expr/src/json_funcs/mod.rs b/native/spark-expr/src/json_funcs/mod.rs new file mode 100644 index 0000000000..de3037590d --- /dev/null +++ b/native/spark-expr/src/json_funcs/mod.rs @@ -0,0 +1,20 @@ +// 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. + +mod to_json; + +pub use to_json::ToJson; diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/json_funcs/to_json.rs similarity index 99% rename from native/spark-expr/src/to_json.rs rename to native/spark-expr/src/json_funcs/to_json.rs index 91b46c6f04..3389ea3a0e 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/json_funcs/to_json.rs @@ -19,7 +19,7 @@ // of the Spark-specific compatibility features that we need (including // being able to specify Spark-compatible cast from all types to string) -use crate::cast::SparkCastOptions; +use crate::SparkCastOptions; use crate::{spark_cast, EvalMode}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; @@ -250,7 +250,7 @@ fn struct_to_json(array: &StructArray, timezone: &str) -> Result { #[cfg(test)] mod test { - use crate::to_json::struct_to_json; + use crate::json_funcs::to_json::struct_to_json; use arrow_array::types::Int32Type; use arrow_array::{Array, PrimitiveArray, StringArray}; use arrow_array::{ArrayRef, BooleanArray, Int32Array, StructArray}; diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index d7313436ee..22bec87ee6 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -23,57 +23,55 @@ mod cast; mod error; mod if_expr; -mod avg; -pub use avg::Avg; mod bitwise_not; pub use bitwise_not::{bitwise_not, BitwiseNotExpr}; -mod avg_decimal; -pub use avg_decimal::AvgDecimal; mod checkoverflow; pub use checkoverflow::CheckOverflow; -mod correlation; -pub use correlation::Correlation; -mod covariance; -pub use covariance::Covariance; -mod strings; -pub use strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}; + mod kernels; mod list; -mod regexp; pub mod scalar_funcs; +mod schema_adapter; +mod static_invoke; +pub use schema_adapter::SparkSchemaAdapterFactory; +pub use static_invoke::*; -pub mod spark_hash; -mod stddev; -pub use stddev::Stddev; -mod structs; -mod sum_decimal; -pub use sum_decimal::SumDecimal; mod negative; +mod struct_funcs; pub use negative::{create_negate_expr, NegativeExpr}; mod normalize_nan; -mod temporal; +mod json_funcs; pub mod test_common; pub mod timezone; -mod to_json; mod unbound; pub use unbound::UnboundColumn; pub mod utils; pub use normalize_nan::NormalizeNaNAndZero; +mod predicate_funcs; +pub use predicate_funcs::{spark_isnan, RLike}; -mod variance; -pub use variance::Variance; +mod agg_funcs; mod comet_scalar_funcs; +pub mod hash_funcs; + +mod string_funcs; + +mod datetime_funcs; +pub use agg_funcs::*; +pub use crate::{CreateNamedStruct, GetStructField}; +pub use crate::{DateTruncExpr, HourExpr, MinuteExpr, SecondExpr, TimestampTruncExpr}; pub use cast::{spark_cast, Cast, SparkCastOptions}; pub use comet_scalar_funcs::create_comet_physical_fun; +pub use datetime_funcs::*; pub use error::{SparkError, SparkResult}; +pub use hash_funcs::*; pub use if_expr::IfExpr; +pub use json_funcs::ToJson; pub use list::{ArrayInsert, GetArrayStructFields, ListExtract}; -pub use regexp::RLike; -pub use structs::{CreateNamedStruct, GetStructField}; -pub use temporal::{DateTruncExpr, HourExpr, MinuteExpr, SecondExpr, TimestampTruncExpr}; -pub use to_json::ToJson; +pub use string_funcs::*; +pub use struct_funcs::*; /// Spark supports three evaluation modes when evaluating expressions, which affect /// the behavior when processing input values that are invalid or would result in an diff --git a/native/spark-expr/src/predicate_funcs/is_nan.rs b/native/spark-expr/src/predicate_funcs/is_nan.rs new file mode 100644 index 0000000000..bf4d7e0f26 --- /dev/null +++ b/native/spark-expr/src/predicate_funcs/is_nan.rs @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{Float32Array, Float64Array}; +use arrow_array::{Array, BooleanArray}; +use arrow_schema::DataType; +use datafusion::physical_plan::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue}; +use std::sync::Arc; + +/// Spark-compatible `isnan` expression +pub fn spark_isnan(args: &[ColumnarValue]) -> Result { + fn set_nulls_to_false(is_nan: BooleanArray) -> ColumnarValue { + match is_nan.nulls() { + Some(nulls) => { + let is_not_null = nulls.inner(); + ColumnarValue::Array(Arc::new(BooleanArray::new( + is_nan.values() & is_not_null, + None, + ))) + } + None => ColumnarValue::Array(Arc::new(is_nan)), + } + } + let value = &args[0]; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float64 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + DataType::Float32 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + other, + ))), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + _ => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + value.data_type(), + ))), + }, + } +} diff --git a/native/spark-expr/src/predicate_funcs/mod.rs b/native/spark-expr/src/predicate_funcs/mod.rs new file mode 100644 index 0000000000..5f1f570c05 --- /dev/null +++ b/native/spark-expr/src/predicate_funcs/mod.rs @@ -0,0 +1,22 @@ +// 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. + +mod is_nan; +mod rlike; + +pub use is_nan::spark_isnan; +pub use rlike::RLike; diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/predicate_funcs/rlike.rs similarity index 100% rename from native/spark-expr/src/regexp.rs rename to native/spark-expr/src/predicate_funcs/rlike.rs diff --git a/native/spark-expr/src/scalar_funcs.rs b/native/spark-expr/src/scalar_funcs.rs index 2961f038dc..52ece10e8d 100644 --- a/native/spark-expr/src/scalar_funcs.rs +++ b/native/spark-expr/src/scalar_funcs.rs @@ -15,45 +15,39 @@ // specific language governing permissions and limitations // under the License. -use arrow::compute::kernels::numeric::{add, sub}; -use arrow::datatypes::IntervalDayTime; use arrow::{ array::{ ArrayRef, AsArray, Decimal128Builder, Float32Array, Float64Array, Int16Array, Int32Array, - Int64Array, Int64Builder, Int8Array, OffsetSizeTrait, + Int64Array, Int64Builder, Int8Array, }, + compute::kernels::numeric::{add, sub}, datatypes::{validate_decimal_precision, Decimal128Type, Int64Type}, }; -use arrow_array::builder::{GenericStringBuilder, IntervalDayTimeBuilder}; -use arrow_array::types::{Int16Type, Int32Type, Int8Type}; +use arrow_array::builder::IntervalDayTimeBuilder; +use arrow_array::types::{Int16Type, Int32Type, Int8Type, IntervalDayTime}; use arrow_array::{Array, ArrowNativeTypeOp, BooleanArray, Datum, Decimal128Array}; use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION}; use datafusion::physical_expr_common::datum; use datafusion::{functions::math::round::round, physical_plan::ColumnarValue}; use datafusion_common::{ - cast::as_generic_string_array, exec_err, internal_err, DataFusionError, - Result as DataFusionResult, ScalarValue, + exec_err, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; use num::{ integer::{div_ceil, div_floor}, BigInt, Signed, ToPrimitive, }; -use std::fmt::Write; use std::{cmp::min, sync::Arc}; mod unhex; pub use unhex::spark_unhex; mod hex; +pub(crate) use hex::hex_strings; pub use hex::spark_hex; mod chr; pub use chr::SparkChrFunc; -pub mod hash_expressions; -// exposed for benchmark only -pub use hash_expressions::{spark_murmur3_hash, spark_xxhash64}; - #[inline] fn get_precision_scale(data_type: &DataType) -> (u8, i8) { let DataType::Decimal128(precision, scale) = data_type else { @@ -390,57 +384,6 @@ pub fn spark_round( } } -/// Similar to DataFusion `rpad`, but not to truncate when the string is already longer than length -pub fn spark_read_side_padding(args: &[ColumnarValue]) -> Result { - match args { - [ColumnarValue::Array(array), ColumnarValue::Scalar(ScalarValue::Int32(Some(length)))] => { - match array.data_type() { - DataType::Utf8 => spark_read_side_padding_internal::(array, *length), - DataType::LargeUtf8 => spark_read_side_padding_internal::(array, *length), - // TODO: handle Dictionary types - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {other:?} for function read_side_padding", - ))), - } - } - other => Err(DataFusionError::Internal(format!( - "Unsupported arguments {other:?} for function read_side_padding", - ))), - } -} - -fn spark_read_side_padding_internal( - array: &ArrayRef, - length: i32, -) -> Result { - let string_array = as_generic_string_array::(array)?; - let length = 0.max(length) as usize; - let space_string = " ".repeat(length); - - let mut builder = - GenericStringBuilder::::with_capacity(string_array.len(), string_array.len() * length); - - for string in string_array.iter() { - match string { - Some(string) => { - // It looks Spark's UTF8String is closer to chars rather than graphemes - // https://stackoverflow.com/a/46290728 - let char_len = string.chars().count(); - if length <= char_len { - builder.append_value(string); - } else { - // write_str updates only the value buffer, not null nor offset buffer - // This is convenient for concatenating str(s) - builder.write_str(string)?; - builder.append_value(&space_string[char_len..]); - } - } - _ => builder.append_null(), - } - } - Ok(ColumnarValue::Array(Arc::new(builder.finish()))) -} - // Let Decimal(p3, s3) as return type i.e. Decimal(p1, s1) / Decimal(p2, s2) = Decimal(p3, s3). // Conversely, Decimal(p1, s1) = Decimal(p2, s2) * Decimal(p3, s3). This means that, in order to // get enough scale that matches with Spark behavior, it requires to widen s1 to s2 + s3 + 1. Since @@ -505,53 +448,6 @@ pub fn spark_decimal_div( Ok(ColumnarValue::Array(Arc::new(result))) } -/// Spark-compatible `isnan` expression -pub fn spark_isnan(args: &[ColumnarValue]) -> Result { - fn set_nulls_to_false(is_nan: BooleanArray) -> ColumnarValue { - match is_nan.nulls() { - Some(nulls) => { - let is_not_null = nulls.inner(); - ColumnarValue::Array(Arc::new(BooleanArray::new( - is_nan.values() & is_not_null, - None, - ))) - } - None => ColumnarValue::Array(Arc::new(is_nan)), - } - } - let value = &args[0]; - match value { - ColumnarValue::Array(array) => match array.data_type() { - DataType::Float64 => { - let array = array.as_any().downcast_ref::().unwrap(); - let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); - Ok(set_nulls_to_false(is_nan)) - } - DataType::Float32 => { - let array = array.as_any().downcast_ref::().unwrap(); - let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); - Ok(set_nulls_to_false(is_nan)) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function isnan", - other, - ))), - }, - ColumnarValue::Scalar(a) => match a { - ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( - a.map(|x| x.is_nan()).unwrap_or(false), - )))), - ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( - a.map(|x| x.is_nan()).unwrap_or(false), - )))), - _ => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function isnan", - value.data_type(), - ))), - }, - } -} - macro_rules! scalar_date_arithmetic { ($start:expr, $days:expr, $op:expr) => {{ let interval = IntervalDayTime::new(*$days as i32, 0); @@ -624,3 +520,50 @@ pub fn spark_date_add(args: &[ColumnarValue]) -> Result Result { spark_date_arithmetic(args, sub) } + +/// Spark-compatible `isnan` expression +pub fn spark_isnan(args: &[ColumnarValue]) -> Result { + fn set_nulls_to_false(is_nan: BooleanArray) -> ColumnarValue { + match is_nan.nulls() { + Some(nulls) => { + let is_not_null = nulls.inner(); + ColumnarValue::Array(Arc::new(BooleanArray::new( + is_nan.values() & is_not_null, + None, + ))) + } + None => ColumnarValue::Array(Arc::new(is_nan)), + } + } + let value = &args[0]; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float64 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + DataType::Float32 => { + let array = array.as_any().downcast_ref::().unwrap(); + let is_nan = BooleanArray::from_unary(array, |x| x.is_nan()); + Ok(set_nulls_to_false(is_nan)) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + other, + ))), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some( + a.map(|x| x.is_nan()).unwrap_or(false), + )))), + _ => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function isnan", + value.data_type(), + ))), + }, + } +} diff --git a/native/spark-expr/src/scalar_funcs/hash_expressions.rs b/native/spark-expr/src/scalar_funcs/hash_expressions.rs deleted file mode 100644 index af423677b7..0000000000 --- a/native/spark-expr/src/scalar_funcs/hash_expressions.rs +++ /dev/null @@ -1,166 +0,0 @@ -// 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. - -use crate::scalar_funcs::hex::hex_strings; -use crate::spark_hash::{create_murmur3_hashes, create_xxhash64_hashes}; - -use arrow_array::{Array, ArrayRef, Int32Array, Int64Array, StringArray}; -use datafusion::functions::crypto::{sha224, sha256, sha384, sha512}; -use datafusion_common::cast::as_binary_array; -use datafusion_common::{exec_err, internal_err, DataFusionError, ScalarValue}; -use datafusion_expr::{ColumnarValue, ScalarUDF}; -use std::sync::Arc; - -/// Spark compatible murmur3 hash (just `hash` in Spark) in vectorized execution fashion -pub fn spark_murmur3_hash(args: &[ColumnarValue]) -> Result { - let length = args.len(); - let seed = &args[length - 1]; - match seed { - ColumnarValue::Scalar(ScalarValue::Int32(Some(seed))) => { - // iterate over the arguments to find out the length of the array - let num_rows = args[0..args.len() - 1] - .iter() - .find_map(|arg| match arg { - ColumnarValue::Array(array) => Some(array.len()), - ColumnarValue::Scalar(_) => None, - }) - .unwrap_or(1); - let mut hashes: Vec = vec![0_u32; num_rows]; - hashes.fill(*seed as u32); - let arrays = args[0..args.len() - 1] - .iter() - .map(|arg| match arg { - ColumnarValue::Array(array) => Arc::clone(array), - ColumnarValue::Scalar(scalar) => { - scalar.clone().to_array_of_size(num_rows).unwrap() - } - }) - .collect::>(); - create_murmur3_hashes(&arrays, &mut hashes)?; - if num_rows == 1 { - Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some( - hashes[0] as i32, - )))) - } else { - let hashes: Vec = hashes.into_iter().map(|x| x as i32).collect(); - Ok(ColumnarValue::Array(Arc::new(Int32Array::from(hashes)))) - } - } - _ => { - internal_err!( - "The seed of function murmur3_hash must be an Int32 scalar value, but got: {:?}.", - seed - ) - } - } -} - -/// Spark compatible xxhash64 in vectorized execution fashion -pub fn spark_xxhash64(args: &[ColumnarValue]) -> Result { - let length = args.len(); - let seed = &args[length - 1]; - match seed { - ColumnarValue::Scalar(ScalarValue::Int64(Some(seed))) => { - // iterate over the arguments to find out the length of the array - let num_rows = args[0..args.len() - 1] - .iter() - .find_map(|arg| match arg { - ColumnarValue::Array(array) => Some(array.len()), - ColumnarValue::Scalar(_) => None, - }) - .unwrap_or(1); - let mut hashes: Vec = vec![0_u64; num_rows]; - hashes.fill(*seed as u64); - let arrays = args[0..args.len() - 1] - .iter() - .map(|arg| match arg { - ColumnarValue::Array(array) => Arc::clone(array), - ColumnarValue::Scalar(scalar) => { - scalar.clone().to_array_of_size(num_rows).unwrap() - } - }) - .collect::>(); - create_xxhash64_hashes(&arrays, &mut hashes)?; - if num_rows == 1 { - Ok(ColumnarValue::Scalar(ScalarValue::Int64(Some( - hashes[0] as i64, - )))) - } else { - let hashes: Vec = hashes.into_iter().map(|x| x as i64).collect(); - Ok(ColumnarValue::Array(Arc::new(Int64Array::from(hashes)))) - } - } - _ => { - internal_err!( - "The seed of function xxhash64 must be an Int64 scalar value, but got: {:?}.", - seed - ) - } - } -} - -/// `sha224` function that simulates Spark's `sha2` expression with bit width 224 -pub fn spark_sha224(args: &[ColumnarValue]) -> Result { - wrap_digest_result_as_hex_string(args, sha224()) -} - -/// `sha256` function that simulates Spark's `sha2` expression with bit width 0 or 256 -pub fn spark_sha256(args: &[ColumnarValue]) -> Result { - wrap_digest_result_as_hex_string(args, sha256()) -} - -/// `sha384` function that simulates Spark's `sha2` expression with bit width 384 -pub fn spark_sha384(args: &[ColumnarValue]) -> Result { - wrap_digest_result_as_hex_string(args, sha384()) -} - -/// `sha512` function that simulates Spark's `sha2` expression with bit width 512 -pub fn spark_sha512(args: &[ColumnarValue]) -> Result { - wrap_digest_result_as_hex_string(args, sha512()) -} - -// Spark requires hex string as the result of sha2 functions, we have to wrap the -// result of digest functions as hex string -fn wrap_digest_result_as_hex_string( - args: &[ColumnarValue], - digest: Arc, -) -> Result { - let row_count = match &args[0] { - ColumnarValue::Array(array) => array.len(), - ColumnarValue::Scalar(_) => 1, - }; - let value = digest.invoke_batch(args, row_count)?; - match value { - ColumnarValue::Array(array) => { - let binary_array = as_binary_array(&array)?; - let string_array: StringArray = binary_array - .iter() - .map(|opt| opt.map(hex_strings::<_>)) - .collect(); - Ok(ColumnarValue::Array(Arc::new(string_array))) - } - ColumnarValue::Scalar(ScalarValue::Binary(opt)) => Ok(ColumnarValue::Scalar( - ScalarValue::Utf8(opt.map(hex_strings::<_>)), - )), - _ => { - exec_err!( - "digest function should return binary value, but got: {:?}", - value.data_type() - ) - } - } -} diff --git a/native/spark-expr/src/scalar_funcs/hex.rs b/native/spark-expr/src/scalar_funcs/hex.rs index e572ba5ef3..4ccd4f4538 100644 --- a/native/spark-expr/src/scalar_funcs/hex.rs +++ b/native/spark-expr/src/scalar_funcs/hex.rs @@ -52,7 +52,7 @@ fn hex_encode>(data: T, lower_case: bool) -> String { } #[inline(always)] -pub(super) fn hex_strings>(data: T) -> String { +pub(crate) fn hex_strings>(data: T) -> String { hex_encode(data, true) } diff --git a/native/spark-expr/src/schema_adapter.rs b/native/spark-expr/src/schema_adapter.rs new file mode 100644 index 0000000000..161ad6f164 --- /dev/null +++ b/native/spark-expr/src/schema_adapter.rs @@ -0,0 +1,376 @@ +// 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. + +//! Custom schema adapter that uses Spark-compatible casts + +use crate::cast::cast_supported; +use crate::{spark_cast, SparkCastOptions}; +use arrow_array::{new_null_array, Array, RecordBatch, RecordBatchOptions}; +use arrow_schema::{Schema, SchemaRef}; +use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper}; +use datafusion_common::plan_err; +use datafusion_expr::ColumnarValue; +use std::sync::Arc; + +/// An implementation of DataFusion's `SchemaAdapterFactory` that uses a Spark-compatible +/// `cast` implementation. +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapterFactory { + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SparkSchemaAdapterFactory { + pub fn new(options: SparkCastOptions) -> Self { + Self { + cast_options: options, + } + } +} + +impl SchemaAdapterFactory for SparkSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema. + /// + /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with + /// the same schema for both the projected table schema and the table + /// schema. + fn create( + &self, + required_schema: SchemaRef, + table_schema: SchemaRef, + ) -> Box { + Box::new(SparkSchemaAdapter { + required_schema, + table_schema, + cast_options: self.cast_options.clone(), + }) + } +} + +/// This SchemaAdapter requires both the table schema and the projected table +/// schema. See [`SchemaMapping`] for more details +#[derive(Clone, Debug)] +pub struct SparkSchemaAdapter { + /// The schema for the table, projected to include only the fields being output (projected) by the + /// associated ParquetExec + required_schema: SchemaRef, + /// The entire table schema for the table we're using this to adapt. + /// + /// This is used to evaluate any filters pushed down into the scan + /// which may refer to columns that are not referred to anywhere + /// else in the plan. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaAdapter for SparkSchemaAdapter { + /// Map a column index in the table schema to a column index in a particular + /// file schema + /// + /// Panics if index is not in range for the table schema + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.required_schema.field(index); + Some(file_schema.fields.find(field.name())?.0) + } + + /// Creates a `SchemaMapping` for casting or mapping the columns from the + /// file schema to the table schema. + /// + /// If the provided `file_schema` contains columns of a different type to + /// the expected `table_schema`, the method will attempt to cast the array + /// data from the file schema to the table schema where possible. + /// + /// Returns a [`SchemaMapping`] that can be applied to the output batch + /// along with an ordered list of columns to project from the file + fn map_schema( + &self, + file_schema: &Schema, + ) -> datafusion_common::Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + let mut field_mappings = vec![None; self.required_schema.fields().len()]; + + for (file_idx, file_field) in file_schema.fields.iter().enumerate() { + if let Some((table_idx, table_field)) = + self.required_schema.fields().find(file_field.name()) + { + if cast_supported( + file_field.data_type(), + table_field.data_type(), + &self.cast_options, + ) { + field_mappings[table_idx] = Some(projection.len()); + projection.push(file_idx); + } else { + return plan_err!( + "Cannot cast file schema field {} of type {:?} to required schema field of type {:?}", + file_field.name(), + file_field.data_type(), + table_field.data_type() + ); + } + } + } + + Ok(( + Arc::new(SchemaMapping { + required_schema: Arc::::clone(&self.required_schema), + field_mappings, + table_schema: Arc::::clone(&self.table_schema), + cast_options: self.cast_options.clone(), + }), + projection, + )) + } +} + +// TODO SchemaMapping is mostly copied from DataFusion but calls spark_cast +// instead of arrow cast - can we reduce the amount of code copied here and make +// the DataFusion version more extensible? + +/// The SchemaMapping struct holds a mapping from the file schema to the table +/// schema and any necessary type conversions. +/// +/// Note, because `map_batch` and `map_partial_batch` functions have different +/// needs, this struct holds two schemas: +/// +/// 1. The projected **table** schema +/// 2. The full table schema +/// +/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which +/// has the projected schema, since that's the schema which is supposed to come +/// out of the execution of this query. Thus `map_batch` uses +/// `projected_table_schema` as it can only operate on the projected fields. +/// +/// [`map_partial_batch`] is used to create a RecordBatch with a schema that +/// can be used for Parquet predicate pushdown, meaning that it may contain +/// fields which are not in the projected schema (as the fields that parquet +/// pushdown filters operate can be completely distinct from the fields that are +/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// `table_schema` to create the resulting RecordBatch (as it could be operating +/// on any fields in the schema). +/// +/// [`map_batch`]: Self::map_batch +/// [`map_partial_batch`]: Self::map_partial_batch +#[derive(Debug)] +pub struct SchemaMapping { + /// The schema of the table. This is the expected schema after conversion + /// and it should match the schema of the query result. + required_schema: SchemaRef, + /// Mapping from field index in `projected_table_schema` to index in + /// projected file_schema. + /// + /// They are Options instead of just plain `usize`s because the table could + /// have fields that don't exist in the file. + field_mappings: Vec>, + /// The entire table schema, as opposed to the projected_table_schema (which + /// only contains the columns that we are projecting out of this query). + /// This contains all fields in the table, regardless of if they will be + /// projected out or not. + table_schema: SchemaRef, + /// Spark cast options + cast_options: SparkCastOptions, +} + +impl SchemaMapper for SchemaMapping { + /// Adapts a `RecordBatch` to match the `projected_table_schema` using the stored mapping and + /// conversions. The produced RecordBatch has a schema that contains only the projected + /// columns, so if one needs a RecordBatch with a schema that references columns which are not + /// in the projected, it would be better to use `map_partial_batch` + fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_rows = batch.num_rows(); + let batch_cols = batch.columns().to_vec(); + + let cols = self + .required_schema + // go through each field in the projected schema + .fields() + .iter() + // and zip it with the index that maps fields from the projected table schema to the + // projected file schema in `batch` + .zip(&self.field_mappings) + // and for each one... + .map(|(field, file_idx)| { + file_idx.map_or_else( + // If this field only exists in the table, and not in the file, then we know + // that it's null, so just return that. + || Ok(new_null_array(field.data_type(), batch_rows)), + // However, if it does exist in both, then try to cast it to the correct output + // type + |batch_idx| { + spark_cast( + ColumnarValue::Array(Arc::clone(&batch_cols[batch_idx])), + field.data_type(), + &self.cast_options, + )? + .into_array(batch_rows) + }, + ) + }) + .collect::, _>>()?; + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::::clone(&self.required_schema); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } + + /// Adapts a [`RecordBatch`]'s schema into one that has all the correct output types and only + /// contains the fields that exist in both the file schema and table schema. + /// + /// Unlike `map_batch` this method also preserves the columns that + /// may not appear in the final output (`projected_table_schema`) but may + /// appear in push down predicates + fn map_partial_batch(&self, batch: RecordBatch) -> datafusion_common::Result { + let batch_cols = batch.columns().to_vec(); + let schema = batch.schema(); + + // for each field in the batch's schema (which is based on a file, not a table)... + let (cols, fields) = schema + .fields() + .iter() + .zip(batch_cols.iter()) + .flat_map(|(field, batch_col)| { + self.table_schema + // try to get the same field from the table schema that we have stored in self + .field_with_name(field.name()) + // and if we don't have it, that's fine, ignore it. This may occur when we've + // created an external table whose fields are a subset of the fields in this + // file, then tried to read data from the file into this table. If that is the + // case here, it's fine to ignore because we don't care about this field + // anyways + .ok() + // but if we do have it, + .map(|table_field| { + // try to cast it into the correct output type. we don't want to ignore this + // error, though, so it's propagated. + spark_cast( + ColumnarValue::Array(Arc::clone(batch_col)), + table_field.data_type(), + &self.cast_options, + )? + .into_array(batch_col.len()) + // and if that works, return the field and column. + .map(|new_col| (new_col, table_field.clone())) + }) + }) + .collect::, _>>()? + .into_iter() + .unzip::<_, _, Vec<_>, Vec<_>>(); + + // Necessary to handle empty batches + let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); + + let schema = Arc::new(Schema::new_with_metadata(fields, schema.metadata().clone())); + let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; + Ok(record_batch) + } +} + +#[cfg(test)] +mod test { + use crate::test_common::file_util::get_temp_filename; + use crate::{EvalMode, SparkCastOptions, SparkSchemaAdapterFactory}; + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use arrow_array::UInt32Array; + use arrow_schema::SchemaRef; + use datafusion::datasource::listing::PartitionedFile; + use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; + use datafusion::execution::object_store::ObjectStoreUrl; + use datafusion::execution::TaskContext; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_common::DataFusionError; + use futures::StreamExt; + use parquet::arrow::ArrowWriter; + use std::fs::File; + use std::sync::Arc; + + #[tokio::test] + async fn parquet_roundtrip_int_as_string() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let ids = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let names = Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])) + as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids, names])?; + + let required_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + ])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + #[tokio::test] + async fn parquet_roundtrip_unsigned_int() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::UInt32, false)])); + + let ids = Arc::new(UInt32Array::from(vec![1, 2, 3])) as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids])?; + + let required_schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let _ = roundtrip(&batch, required_schema).await?; + + Ok(()) + } + + /// Create a Parquet file containing a single batch and then read the batch back using + /// the specified required_schema. This will cause the SchemaAdapter code to be used. + async fn roundtrip( + batch: &RecordBatch, + required_schema: SchemaRef, + ) -> Result { + let filename = get_temp_filename(); + let filename = filename.as_path().as_os_str().to_str().unwrap().to_string(); + let file = File::create(&filename)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&batch.schema()), None)?; + writer.write(batch)?; + writer.close()?; + + let object_store_url = ObjectStoreUrl::local_filesystem(); + let file_scan_config = FileScanConfig::new(object_store_url, required_schema) + .with_file_groups(vec![vec![PartitionedFile::from_path( + filename.to_string(), + )?]]); + + let mut spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); + spark_cast_options.allow_cast_unsigned_ints = true; + + let parquet_exec = ParquetExec::builder(file_scan_config) + .with_schema_adapter_factory(Arc::new(SparkSchemaAdapterFactory::new( + spark_cast_options, + ))) + .build(); + + let mut stream = parquet_exec + .execute(0, Arc::new(TaskContext::default())) + .unwrap(); + stream.next().await.unwrap() + } +} diff --git a/native/spark-expr/src/spark_hash.rs b/native/spark-expr/src/spark_hash.rs deleted file mode 100644 index 1402f71715..0000000000 --- a/native/spark-expr/src/spark_hash.rs +++ /dev/null @@ -1,712 +0,0 @@ -// 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. - -//! This includes utilities for hashing and murmur3 hashing. - -use arrow::{ - compute::take, - datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}, -}; -use std::sync::Arc; -use twox_hash::XxHash64; - -use datafusion::{ - arrow::{ - array::*, - datatypes::{ - ArrowDictionaryKeyType, ArrowNativeType, DataType, Int16Type, Int32Type, Int64Type, - Int8Type, TimeUnit, - }, - }, - error::{DataFusionError, Result}, -}; - -#[inline] -pub(crate) fn spark_compatible_xxhash64>(data: T, seed: u64) -> u64 { - XxHash64::oneshot(seed, data.as_ref()) -} - -/// Spark-compatible murmur3 hash function -#[inline] -pub fn spark_compatible_murmur3_hash>(data: T, seed: u32) -> u32 { - #[inline] - fn mix_k1(mut k1: i32) -> i32 { - k1 = k1.mul_wrapping(0xcc9e2d51u32 as i32); - k1 = k1.rotate_left(15); - k1 = k1.mul_wrapping(0x1b873593u32 as i32); - k1 - } - - #[inline] - fn mix_h1(mut h1: i32, k1: i32) -> i32 { - h1 ^= k1; - h1 = h1.rotate_left(13); - h1 = h1.mul_wrapping(5).add_wrapping(0xe6546b64u32 as i32); - h1 - } - - #[inline] - fn fmix(mut h1: i32, len: i32) -> i32 { - h1 ^= len; - h1 ^= (h1 as u32 >> 16) as i32; - h1 = h1.mul_wrapping(0x85ebca6bu32 as i32); - h1 ^= (h1 as u32 >> 13) as i32; - h1 = h1.mul_wrapping(0xc2b2ae35u32 as i32); - h1 ^= (h1 as u32 >> 16) as i32; - h1 - } - - #[inline] - unsafe fn hash_bytes_by_int(data: &[u8], seed: u32) -> i32 { - // safety: data length must be aligned to 4 bytes - let mut h1 = seed as i32; - for i in (0..data.len()).step_by(4) { - let ints = data.as_ptr().add(i) as *const i32; - let mut half_word = ints.read_unaligned(); - if cfg!(target_endian = "big") { - half_word = half_word.reverse_bits(); - } - h1 = mix_h1(h1, mix_k1(half_word)); - } - h1 - } - let data = data.as_ref(); - let len = data.len(); - let len_aligned = len - len % 4; - - // safety: - // avoid boundary checking in performance critical codes. - // all operations are guaranteed to be safe - // data is &[u8] so we do not need to check for proper alignment - unsafe { - let mut h1 = if len_aligned > 0 { - hash_bytes_by_int(&data[0..len_aligned], seed) - } else { - seed as i32 - }; - - for i in len_aligned..len { - let half_word = *data.get_unchecked(i) as i8 as i32; - h1 = mix_h1(h1, mix_k1(half_word)); - } - fmix(h1, len as i32) as u32 - } -} - -macro_rules! hash_array { - ($array_type: ident, $column: ident, $hashes: ident, $hash_method: ident) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - if array.null_count() == 0 { - for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = $hash_method(&array.value(i), *hash); - } - } else { - for (i, hash) in $hashes.iter_mut().enumerate() { - if !array.is_null(i) { - *hash = $hash_method(&array.value(i), *hash); - } - } - } - }; -} - -macro_rules! hash_array_boolean { - ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident, $hash_method: ident) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - if array.null_count() == 0 { - for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); - } - } else { - for (i, hash) in $hashes.iter_mut().enumerate() { - if !array.is_null(i) { - *hash = - $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); - } - } - } - }; -} - -macro_rules! hash_array_primitive { - ($array_type: ident, $column: ident, $ty: ident, $hashes: ident, $hash_method: ident) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - let values = array.values(); - - if array.null_count() == 0 { - for (hash, value) in $hashes.iter_mut().zip(values.iter()) { - *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); - } - } else { - for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { - if !array.is_null(i) { - *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); - } - } - } - }; -} - -macro_rules! hash_array_primitive_float { - ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident, $hash_method: ident) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - let values = array.values(); - - if array.null_count() == 0 { - for (hash, value) in $hashes.iter_mut().zip(values.iter()) { - // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. - if *value == 0.0 && value.is_sign_negative() { - *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); - } else { - *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); - } - } - } else { - for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { - if !array.is_null(i) { - // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. - if *value == 0.0 && value.is_sign_negative() { - *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); - } else { - *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); - } - } - } - } - }; -} - -macro_rules! hash_array_decimal { - ($array_type:ident, $column: ident, $hashes: ident, $hash_method: ident) => { - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - - if array.null_count() == 0 { - for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = $hash_method(array.value(i).to_le_bytes(), *hash); - } - } else { - for (i, hash) in $hashes.iter_mut().enumerate() { - if !array.is_null(i) { - *hash = $hash_method(array.value(i).to_le_bytes(), *hash); - } - } - } - }; -} - -/// Hash the values in a dictionary array -fn create_hashes_dictionary( - array: &ArrayRef, - hashes_buffer: &mut [u32], - first_col: bool, -) -> Result<()> { - let dict_array = array.as_any().downcast_ref::>().unwrap(); - if !first_col { - // unpack the dictionary array as each row may have a different hash input - let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; - create_murmur3_hashes(&[unpacked], hashes_buffer)?; - } else { - // For the first column, hash each dictionary value once, and then use - // that computed hash for each key value to avoid a potentially - // expensive redundant hashing for large dictionary elements (e.g. strings) - let dict_values = Arc::clone(dict_array.values()); - // same initial seed as Spark - let mut dict_hashes = vec![42; dict_values.len()]; - create_murmur3_hashes(&[dict_values], &mut dict_hashes)?; - for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { - if let Some(key) = key { - let idx = key.to_usize().ok_or_else(|| { - DataFusionError::Internal(format!( - "Can not convert key value {:?} to usize in dictionary of type {:?}", - key, - dict_array.data_type() - )) - })?; - *hash = dict_hashes[idx] - } // no update for Null, consistent with other hashes - } - } - Ok(()) -} - -// Hash the values in a dictionary array using xxhash64 -fn create_xxhash64_hashes_dictionary( - array: &ArrayRef, - hashes_buffer: &mut [u64], - first_col: bool, -) -> Result<()> { - let dict_array = array.as_any().downcast_ref::>().unwrap(); - if !first_col { - let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; - create_xxhash64_hashes(&[unpacked], hashes_buffer)?; - } else { - // Hash each dictionary value once, and then use that computed - // hash for each key value to avoid a potentially expensive - // redundant hashing for large dictionary elements (e.g. strings) - let dict_values = Arc::clone(dict_array.values()); - // same initial seed as Spark - let mut dict_hashes = vec![42u64; dict_values.len()]; - create_xxhash64_hashes(&[dict_values], &mut dict_hashes)?; - - for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { - if let Some(key) = key { - let idx = key.to_usize().ok_or_else(|| { - DataFusionError::Internal(format!( - "Can not convert key value {:?} to usize in dictionary of type {:?}", - key, - dict_array.data_type() - )) - })?; - *hash = dict_hashes[idx] - } // no update for Null, consistent with other hashes - } - } - Ok(()) -} - -/// Creates hash values for every row, based on the values in the -/// columns. -/// -/// The number of rows to hash is determined by `hashes_buffer.len()`. -/// `hashes_buffer` should be pre-sized appropriately -/// -/// `hash_method` is the hash function to use. -/// `create_dictionary_hash_method` is the function to create hashes for dictionary arrays input. -macro_rules! create_hashes_internal { - ($arrays: ident, $hashes_buffer: ident, $hash_method: ident, $create_dictionary_hash_method: ident) => { - for (i, col) in $arrays.iter().enumerate() { - let first_col = i == 0; - match col.data_type() { - DataType::Boolean => { - hash_array_boolean!(BooleanArray, col, i32, $hashes_buffer, $hash_method); - } - DataType::Int8 => { - hash_array_primitive!(Int8Array, col, i32, $hashes_buffer, $hash_method); - } - DataType::Int16 => { - hash_array_primitive!(Int16Array, col, i32, $hashes_buffer, $hash_method); - } - DataType::Int32 => { - hash_array_primitive!(Int32Array, col, i32, $hashes_buffer, $hash_method); - } - DataType::Int64 => { - hash_array_primitive!(Int64Array, col, i64, $hashes_buffer, $hash_method); - } - DataType::Float32 => { - hash_array_primitive_float!( - Float32Array, - col, - f32, - i32, - $hashes_buffer, - $hash_method - ); - } - DataType::Float64 => { - hash_array_primitive_float!( - Float64Array, - col, - f64, - i64, - $hashes_buffer, - $hash_method - ); - } - DataType::Timestamp(TimeUnit::Second, _) => { - hash_array_primitive!( - TimestampSecondArray, - col, - i64, - $hashes_buffer, - $hash_method - ); - } - DataType::Timestamp(TimeUnit::Millisecond, _) => { - hash_array_primitive!( - TimestampMillisecondArray, - col, - i64, - $hashes_buffer, - $hash_method - ); - } - DataType::Timestamp(TimeUnit::Microsecond, _) => { - hash_array_primitive!( - TimestampMicrosecondArray, - col, - i64, - $hashes_buffer, - $hash_method - ); - } - DataType::Timestamp(TimeUnit::Nanosecond, _) => { - hash_array_primitive!( - TimestampNanosecondArray, - col, - i64, - $hashes_buffer, - $hash_method - ); - } - DataType::Date32 => { - hash_array_primitive!(Date32Array, col, i32, $hashes_buffer, $hash_method); - } - DataType::Date64 => { - hash_array_primitive!(Date64Array, col, i64, $hashes_buffer, $hash_method); - } - DataType::Utf8 => { - hash_array!(StringArray, col, $hashes_buffer, $hash_method); - } - DataType::LargeUtf8 => { - hash_array!(LargeStringArray, col, $hashes_buffer, $hash_method); - } - DataType::Binary => { - hash_array!(BinaryArray, col, $hashes_buffer, $hash_method); - } - DataType::LargeBinary => { - hash_array!(LargeBinaryArray, col, $hashes_buffer, $hash_method); - } - DataType::FixedSizeBinary(_) => { - hash_array!(FixedSizeBinaryArray, col, $hashes_buffer, $hash_method); - } - DataType::Decimal128(_, _) => { - hash_array_decimal!(Decimal128Array, col, $hashes_buffer, $hash_method); - } - DataType::Dictionary(index_type, _) => match **index_type { - DataType::Int8 => { - $create_dictionary_hash_method::(col, $hashes_buffer, first_col)?; - } - DataType::Int16 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::Int32 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::Int64 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::UInt8 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::UInt16 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::UInt32 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - DataType::UInt64 => { - $create_dictionary_hash_method::( - col, - $hashes_buffer, - first_col, - )?; - } - _ => { - return Err(DataFusionError::Internal(format!( - "Unsupported dictionary type in hasher hashing: {}", - col.data_type(), - ))) - } - }, - _ => { - // This is internal because we should have caught this before. - return Err(DataFusionError::Internal(format!( - "Unsupported data type in hasher: {}", - col.data_type() - ))); - } - } - } - }; -} - -/// Creates hash values for every row, based on the values in the -/// columns. -/// -/// The number of rows to hash is determined by `hashes_buffer.len()`. -/// `hashes_buffer` should be pre-sized appropriately -pub fn create_murmur3_hashes<'a>( - arrays: &[ArrayRef], - hashes_buffer: &'a mut [u32], -) -> Result<&'a mut [u32]> { - create_hashes_internal!( - arrays, - hashes_buffer, - spark_compatible_murmur3_hash, - create_hashes_dictionary - ); - Ok(hashes_buffer) -} - -/// Creates xxhash64 hash values for every row, based on the values in the -/// columns. -/// -/// The number of rows to hash is determined by `hashes_buffer.len()`. -/// `hashes_buffer` should be pre-sized appropriately -pub fn create_xxhash64_hashes<'a>( - arrays: &[ArrayRef], - hashes_buffer: &'a mut [u64], -) -> Result<&'a mut [u64]> { - create_hashes_internal!( - arrays, - hashes_buffer, - spark_compatible_xxhash64, - create_xxhash64_hashes_dictionary - ); - Ok(hashes_buffer) -} - -#[cfg(test)] -mod tests { - use arrow::array::{Float32Array, Float64Array}; - use std::sync::Arc; - - use super::{create_murmur3_hashes, create_xxhash64_hashes}; - use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; - - macro_rules! test_hashes_internal { - ($hash_method: ident, $input: expr, $initial_seeds: expr, $expected: expr) => { - let i = $input; - let mut hashes = $initial_seeds.clone(); - $hash_method(&[i], &mut hashes).unwrap(); - assert_eq!(hashes, $expected); - }; - } - - macro_rules! test_hashes_with_nulls { - ($method: ident, $t: ty, $values: ident, $expected: ident, $seed_type: ty) => { - // copied before inserting nulls - let mut input_with_nulls = $values.clone(); - let mut expected_with_nulls = $expected.clone(); - // test before inserting nulls - let len = $values.len(); - let initial_seeds = vec![42 as $seed_type; len]; - let i = Arc::new(<$t>::from($values)) as ArrayRef; - test_hashes_internal!($method, i, initial_seeds, $expected); - - // test with nulls - let median = len / 2; - input_with_nulls.insert(0, None); - input_with_nulls.insert(median, None); - expected_with_nulls.insert(0, 42 as $seed_type); - expected_with_nulls.insert(median, 42 as $seed_type); - let len_with_nulls = len + 2; - let initial_seeds_with_nulls = vec![42 as $seed_type; len_with_nulls]; - let nullable_input = Arc::new(<$t>::from(input_with_nulls)) as ArrayRef; - test_hashes_internal!( - $method, - nullable_input, - initial_seeds_with_nulls, - expected_with_nulls - ); - }; - } - - fn test_murmur3_hash>> + 'static>( - values: Vec>, - expected: Vec, - ) { - test_hashes_with_nulls!(create_murmur3_hashes, T, values, expected, u32); - } - - fn test_xxhash64_hash>> + 'static>( - values: Vec>, - expected: Vec, - ) { - test_hashes_with_nulls!(create_xxhash64_hashes, T, values, expected, u64); - } - - #[test] - fn test_i8() { - test_murmur3_hash::( - vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365], - ); - test_xxhash64_hash::( - vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![ - 0xa309b38455455929, - 0x3229fbc4681e48f3, - 0x1bfdda8861c06e45, - 0x77cc15d9f9f2cdc2, - 0x39bc22b9e94d81d0, - ], - ); - } - - #[test] - fn test_i32() { - test_murmur3_hash::( - vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6], - ); - test_xxhash64_hash::( - vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], - vec![ - 0xa309b38455455929, - 0x3229fbc4681e48f3, - 0x1bfdda8861c06e45, - 0x14f0ac009c21721c, - 0x1cc7cb8d034769cd, - ], - ); - } - - #[test] - fn test_i64() { - test_murmur3_hash::( - vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], - vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb], - ); - test_xxhash64_hash::( - vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], - vec![ - 0x9ed50fd59358d232, - 0xb71b47ebda15746c, - 0x358ae035bfb46fd2, - 0xd2f1c616ae7eb306, - 0x88608019c494c1f4, - ], - ); - } - - #[test] - fn test_f32() { - test_murmur3_hash::( - vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ], - vec![ - 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, - ], - ); - test_xxhash64_hash::( - vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ], - vec![ - 0x9b92689757fcdbd, - 0x3229fbc4681e48f3, - 0x3229fbc4681e48f3, - 0xa2becc0e61bb3823, - 0x8f20ab82d4f3687f, - 0xdce4982d97f7ac4, - ], - ) - } - - #[test] - fn test_f64() { - test_murmur3_hash::( - vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ], - vec![ - 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, - ], - ); - - test_xxhash64_hash::( - vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - Some(99999999999.99999999999), - Some(-99999999999.99999999999), - ], - vec![ - 0xe1fd6e07fee8ad53, - 0xb71b47ebda15746c, - 0xb71b47ebda15746c, - 0x8cdde022746f8f1f, - 0x793c5c88d313eac7, - 0xc5e60e7b75d9b232, - ], - ) - } - - #[test] - fn test_str() { - let input = [ - "hello", "bar", "", "😁", "天地", "a", "ab", "abc", "abcd", "abcde", - ] - .iter() - .map(|s| Some(s.to_string())) - .collect::>>(); - let expected: Vec = vec![ - 3286402344, 2486176763, 142593372, 885025535, 2395000894, 1485273170, 0xfa37157b, - 1322437556, 0xe860e5cc, 814637928, - ]; - - test_murmur3_hash::(input.clone(), expected); - test_xxhash64_hash::( - input, - vec![ - 0xc3629e6318d53932, - 0xe7097b6a54378d8a, - 0x98b1582b0977e704, - 0xa80d9d5a6a523bd5, - 0xfcba5f61ac666c61, - 0x88e4fe59adf7b0cc, - 0x259dd873209a3fe3, - 0x13c1d910702770e6, - 0xa17b5eb5dc364dff, - 0xf241303e4a90f299, - ], - ) - } -} diff --git a/native/spark-expr/src/static_invoke/char_varchar_utils/mod.rs b/native/spark-expr/src/static_invoke/char_varchar_utils/mod.rs new file mode 100644 index 0000000000..fff6134dab --- /dev/null +++ b/native/spark-expr/src/static_invoke/char_varchar_utils/mod.rs @@ -0,0 +1,20 @@ +// 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. + +mod read_side_padding; + +pub use read_side_padding::spark_read_side_padding; diff --git a/native/spark-expr/src/static_invoke/char_varchar_utils/read_side_padding.rs b/native/spark-expr/src/static_invoke/char_varchar_utils/read_side_padding.rs new file mode 100644 index 0000000000..15807bf57d --- /dev/null +++ b/native/spark-expr/src/static_invoke/char_varchar_utils/read_side_padding.rs @@ -0,0 +1,76 @@ +// 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. + +use arrow::array::{ArrayRef, OffsetSizeTrait}; +use arrow_array::builder::GenericStringBuilder; +use arrow_array::Array; +use arrow_schema::DataType; +use datafusion::physical_plan::ColumnarValue; +use datafusion_common::{cast::as_generic_string_array, DataFusionError, ScalarValue}; +use std::fmt::Write; +use std::sync::Arc; + +/// Similar to DataFusion `rpad`, but not to truncate when the string is already longer than length +pub fn spark_read_side_padding(args: &[ColumnarValue]) -> Result { + match args { + [ColumnarValue::Array(array), ColumnarValue::Scalar(ScalarValue::Int32(Some(length)))] => { + match array.data_type() { + DataType::Utf8 => spark_read_side_padding_internal::(array, *length), + DataType::LargeUtf8 => spark_read_side_padding_internal::(array, *length), + // TODO: handle Dictionary types + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function read_side_padding", + ))), + } + } + other => Err(DataFusionError::Internal(format!( + "Unsupported arguments {other:?} for function read_side_padding", + ))), + } +} + +fn spark_read_side_padding_internal( + array: &ArrayRef, + length: i32, +) -> Result { + let string_array = as_generic_string_array::(array)?; + let length = 0.max(length) as usize; + let space_string = " ".repeat(length); + + let mut builder = + GenericStringBuilder::::with_capacity(string_array.len(), string_array.len() * length); + + for string in string_array.iter() { + match string { + Some(string) => { + // It looks Spark's UTF8String is closer to chars rather than graphemes + // https://stackoverflow.com/a/46290728 + let char_len = string.chars().count(); + if length <= char_len { + builder.append_value(string); + } else { + // write_str updates only the value buffer, not null nor offset buffer + // This is convenient for concatenating str(s) + builder.write_str(string)?; + builder.append_value(&space_string[char_len..]); + } + } + _ => builder.append_null(), + } + } + Ok(ColumnarValue::Array(Arc::new(builder.finish()))) +} diff --git a/native/spark-expr/src/static_invoke/mod.rs b/native/spark-expr/src/static_invoke/mod.rs new file mode 100644 index 0000000000..4072e13b70 --- /dev/null +++ b/native/spark-expr/src/static_invoke/mod.rs @@ -0,0 +1,20 @@ +// 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. + +mod char_varchar_utils; + +pub use char_varchar_utils::spark_read_side_padding; diff --git a/native/spark-expr/src/string_funcs/mod.rs b/native/spark-expr/src/string_funcs/mod.rs new file mode 100644 index 0000000000..2c2a5b37c7 --- /dev/null +++ b/native/spark-expr/src/string_funcs/mod.rs @@ -0,0 +1,24 @@ +// 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. + +mod prediction; +mod string_space; +mod substring; + +pub use prediction::*; +pub use string_space::StringSpaceExpr; +pub use substring::SubstringExpr; diff --git a/native/spark-expr/src/string_funcs/prediction.rs b/native/spark-expr/src/string_funcs/prediction.rs new file mode 100644 index 0000000000..d2ef82fcbe --- /dev/null +++ b/native/spark-expr/src/string_funcs/prediction.rs @@ -0,0 +1,137 @@ +// 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. + +#![allow(deprecated)] + +use arrow::{ + compute::{ + contains_dyn, contains_utf8_scalar_dyn, ends_with_dyn, ends_with_utf8_scalar_dyn, like_dyn, + like_utf8_scalar_dyn, starts_with_dyn, starts_with_utf8_scalar_dyn, + }, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue::Utf8}; +use datafusion_physical_expr::PhysicalExpr; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::Hash, + sync::Arc, +}; + +macro_rules! make_predicate_function { + ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { + #[derive(Debug, Eq)] + pub struct $name { + left: Arc, + right: Arc, + } + + impl $name { + pub fn new(left: Arc, right: Arc) -> Self { + Self { left, right } + } + } + + impl Display for $name { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "$name [left: {}, right: {}]", self.left, self.right) + } + } + + impl Hash for $name { + fn hash(&self, state: &mut H) { + self.left.hash(state); + self.right.hash(state); + } + } + + impl PartialEq for $name { + fn eq(&self, other: &Self) -> bool { + self.left.eq(&other.left) && self.right.eq(&other.right) + } + } + + impl PhysicalExpr for $name { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> datafusion_common::Result { + Ok(DataType::Boolean) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let left_arg = self.left.evaluate(batch)?; + let right_arg = self.right.evaluate(batch)?; + + let array = match (left_arg, right_arg) { + // array (op) scalar + (ColumnarValue::Array(array), ColumnarValue::Scalar(Utf8(Some(string)))) => { + $str_scalar_kernel(&array, string.as_str()) + } + (ColumnarValue::Array(_), ColumnarValue::Scalar(other)) => { + return Err(DataFusionError::Execution(format!( + "Should be String but got: {:?}", + other + ))) + } + // array (op) array + (ColumnarValue::Array(array1), ColumnarValue::Array(array2)) => { + $kernel(&array1, &array2) + } + // scalar (op) scalar should be folded at Spark optimizer + _ => { + return Err(DataFusionError::Execution( + "Predicate on two literals should be folded at Spark".to_string(), + )) + } + }?; + + Ok(ColumnarValue::Array(Arc::new(array))) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.left, &self.right] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new($name::new( + children[0].clone(), + children[1].clone(), + ))) + } + } + }; +} + +make_predicate_function!(Like, like_dyn, like_utf8_scalar_dyn); + +make_predicate_function!(StartsWith, starts_with_dyn, starts_with_utf8_scalar_dyn); + +make_predicate_function!(EndsWith, ends_with_dyn, ends_with_utf8_scalar_dyn); + +make_predicate_function!(Contains, contains_dyn, contains_utf8_scalar_dyn); diff --git a/native/spark-expr/src/string_funcs/string_space.rs b/native/spark-expr/src/string_funcs/string_space.rs new file mode 100644 index 0000000000..db70929057 --- /dev/null +++ b/native/spark-expr/src/string_funcs/string_space.rs @@ -0,0 +1,104 @@ +// 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. + +#![allow(deprecated)] + +use crate::kernels::strings::string_space; +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalExpr; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::Hash, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct StringSpaceExpr { + pub child: Arc, +} + +impl Hash for StringSpaceExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + } +} + +impl PartialEq for StringSpaceExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + } +} + +impl StringSpaceExpr { + pub fn new(child: Arc) -> Self { + Self { child } + } +} + +impl Display for StringSpaceExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "StringSpace [child: {}] ", self.child) + } +} + +impl PhysicalExpr for StringSpaceExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + match self.child.data_type(input_schema)? { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Utf8))) + } + _ => Ok(DataType::Utf8), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let result = string_space(&array)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "StringSpace(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new(StringSpaceExpr::new(Arc::clone(&children[0])))) + } +} diff --git a/native/spark-expr/src/string_funcs/substring.rs b/native/spark-expr/src/string_funcs/substring.rs new file mode 100644 index 0000000000..741ea9139d --- /dev/null +++ b/native/spark-expr/src/string_funcs/substring.rs @@ -0,0 +1,111 @@ +// 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. + +#![allow(deprecated)] + +use crate::kernels::strings::substring; +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalExpr; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::Hash, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct SubstringExpr { + pub child: Arc, + pub start: i64, + pub len: u64, +} + +impl Hash for SubstringExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.start.hash(state); + self.len.hash(state); + } +} + +impl PartialEq for SubstringExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.start.eq(&other.start) && self.len.eq(&other.len) + } +} + +impl SubstringExpr { + pub fn new(child: Arc, start: i64, len: u64) -> Self { + Self { child, start, len } + } +} + +impl Display for SubstringExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "StringSpace [start: {}, len: {}, child: {}]", + self.start, self.len, self.child + ) + } +} + +impl PhysicalExpr for SubstringExpr { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result { + self.child.data_type(input_schema) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let result = substring(&array, self.start, self.len)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Substring(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new(SubstringExpr::new( + Arc::clone(&children[0]), + self.start, + self.len, + ))) + } +} diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/struct_funcs/create_named_struct.rs similarity index 64% rename from native/spark-expr/src/structs.rs rename to native/spark-expr/src/struct_funcs/create_named_struct.rs index 7cc49e4281..df63127412 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/struct_funcs/create_named_struct.rs @@ -16,10 +16,10 @@ // under the License. use arrow::record_batch::RecordBatch; -use arrow_array::{Array, StructArray}; +use arrow_array::StructArray; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; +use datafusion_common::Result as DataFusionResult; use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, @@ -106,102 +106,6 @@ impl Display for CreateNamedStruct { } } -#[derive(Debug, Eq)] -pub struct GetStructField { - child: Arc, - ordinal: usize, -} - -impl Hash for GetStructField { - fn hash(&self, state: &mut H) { - self.child.hash(state); - self.ordinal.hash(state); - } -} -impl PartialEq for GetStructField { - fn eq(&self, other: &Self) -> bool { - self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) - } -} - -impl GetStructField { - pub fn new(child: Arc, ordinal: usize) -> Self { - Self { child, ordinal } - } - - fn child_field(&self, input_schema: &Schema) -> DataFusionResult> { - match self.child.data_type(input_schema)? { - DataType::Struct(fields) => Ok(Arc::clone(&fields[self.ordinal])), - data_type => Err(DataFusionError::Plan(format!( - "Expect struct field, got {:?}", - data_type - ))), - } - } -} - -impl PhysicalExpr for GetStructField { - fn as_any(&self) -> &dyn Any { - self - } - - fn data_type(&self, input_schema: &Schema) -> DataFusionResult { - Ok(self.child_field(input_schema)?.data_type().clone()) - } - - fn nullable(&self, input_schema: &Schema) -> DataFusionResult { - Ok(self.child_field(input_schema)?.is_nullable()) - } - - fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { - let child_value = self.child.evaluate(batch)?; - - match child_value { - ColumnarValue::Array(array) => { - let struct_array = array - .as_any() - .downcast_ref::() - .expect("A struct is expected"); - - Ok(ColumnarValue::Array(Arc::clone( - struct_array.column(self.ordinal), - ))) - } - ColumnarValue::Scalar(ScalarValue::Struct(struct_array)) => Ok(ColumnarValue::Array( - Arc::clone(struct_array.column(self.ordinal)), - )), - value => Err(DataFusionError::Execution(format!( - "Expected a struct array, got {:?}", - value - ))), - } - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.child] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> datafusion_common::Result> { - Ok(Arc::new(GetStructField::new( - Arc::clone(&children[0]), - self.ordinal, - ))) - } -} - -impl Display for GetStructField { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!( - f, - "GetStructField [child: {:?}, ordinal: {:?}]", - self.child, self.ordinal - ) - } -} - #[cfg(test)] mod test { use super::CreateNamedStruct; diff --git a/native/spark-expr/src/struct_funcs/get_struct_field.rs b/native/spark-expr/src/struct_funcs/get_struct_field.rs new file mode 100644 index 0000000000..c4e1a1e239 --- /dev/null +++ b/native/spark-expr/src/struct_funcs/get_struct_field.rs @@ -0,0 +1,125 @@ +// 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. + +use arrow::record_batch::RecordBatch; +use arrow_array::{Array, StructArray}; +use arrow_schema::{DataType, Field, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; +use datafusion_physical_expr::PhysicalExpr; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::Hash, + sync::Arc, +}; + +#[derive(Debug, Eq)] +pub struct GetStructField { + child: Arc, + ordinal: usize, +} + +impl Hash for GetStructField { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + } +} +impl PartialEq for GetStructField { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) + } +} + +impl GetStructField { + pub fn new(child: Arc, ordinal: usize) -> Self { + Self { child, ordinal } + } + + fn child_field(&self, input_schema: &Schema) -> DataFusionResult> { + match self.child.data_type(input_schema)? { + DataType::Struct(fields) => Ok(Arc::clone(&fields[self.ordinal])), + data_type => Err(DataFusionError::Plan(format!( + "Expect struct field, got {:?}", + data_type + ))), + } + } +} + +impl PhysicalExpr for GetStructField { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> DataFusionResult { + Ok(self.child_field(input_schema)?.data_type().clone()) + } + + fn nullable(&self, input_schema: &Schema) -> DataFusionResult { + Ok(self.child_field(input_schema)?.is_nullable()) + } + + fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { + let child_value = self.child.evaluate(batch)?; + + match child_value { + ColumnarValue::Array(array) => { + let struct_array = array + .as_any() + .downcast_ref::() + .expect("A struct is expected"); + + Ok(ColumnarValue::Array(Arc::clone( + struct_array.column(self.ordinal), + ))) + } + ColumnarValue::Scalar(ScalarValue::Struct(struct_array)) => Ok(ColumnarValue::Array( + Arc::clone(struct_array.column(self.ordinal)), + )), + value => Err(DataFusionError::Execution(format!( + "Expected a struct array, got {:?}", + value + ))), + } + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion_common::Result> { + Ok(Arc::new(GetStructField::new( + Arc::clone(&children[0]), + self.ordinal, + ))) + } +} + +impl Display for GetStructField { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "GetStructField [child: {:?}, ordinal: {:?}]", + self.child, self.ordinal + ) + } +} diff --git a/native/spark-expr/src/struct_funcs/mod.rs b/native/spark-expr/src/struct_funcs/mod.rs new file mode 100644 index 0000000000..86edcceac9 --- /dev/null +++ b/native/spark-expr/src/struct_funcs/mod.rs @@ -0,0 +1,22 @@ +// 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. + +mod create_named_struct; +mod get_struct_field; + +pub use create_named_struct::CreateNamedStruct; +pub use get_struct_field::GetStructField; diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index f3648a679f..e71d713f59 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -230,10 +230,8 @@ impl Accumulator for VarianceAccumulator { Ok(ScalarValue::Float64(match self.count { count if count == 0.0 => None, - count if count == 1.0 => { - if let StatsType::Population = self.stats_type { - Some(0.0) - } else if self.null_on_divide_by_zero { + count if count == 1.0 && StatsType::Sample == self.stats_type => { + if self.null_on_divide_by_zero { None } else { Some(f64::NAN) diff --git a/pom.xml b/pom.xml index cdc44a5ca3..76e2288ccc 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ under the License. 3.4.3 3.4 provided - 3.19.6 + 3.21.12 1.13.1 provided 16.0.0 diff --git a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java index 54e349c137..f6e6ca96a7 100644 --- a/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java +++ b/spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java @@ -52,7 +52,7 @@ public static CometShuffleMemoryAllocatorTrait getInstance( (boolean) CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST().get(); - if (isSparkTesting && !useUnifiedMemAllocator) { + if (!useUnifiedMemAllocator) { synchronized (CometShuffleMemoryAllocator.class) { if (INSTANCE == null) { // CometTestShuffleMemoryAllocator handles pages by itself so it can be a singleton. diff --git a/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java b/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java index cc44955705..1e3762a6c4 100644 --- a/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java +++ b/spark/src/main/java/org/apache/spark/shuffle/sort/CometShuffleExternalSorter.java @@ -107,6 +107,8 @@ public final class CometShuffleExternalSorter implements CometShuffleChecksumSup private final long[] partitionChecksums; private final String checksumAlgorithm; + private final String compressionCodec; + private final int compressionLevel; // The memory allocator for this sorter. It is used to allocate/free memory pages for this sorter. // Because we need to allocate off-heap memory regardless of configured Spark memory mode @@ -153,6 +155,9 @@ public CometShuffleExternalSorter( this.peakMemoryUsedBytes = getMemoryUsage(); this.partitionChecksums = createPartitionChecksums(numPartitions, conf); this.checksumAlgorithm = getChecksumAlgorithm(conf); + this.compressionCodec = CometConf$.MODULE$.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC().get(); + this.compressionLevel = + (int) CometConf$.MODULE$.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL().get(); this.initialSize = initialSize; @@ -556,7 +561,9 @@ public void writeSortedFileNative(boolean isLastFile) throws IOException { spillInfo.file, rowPartition, writeMetricsToUse, - preferDictionaryRatio); + preferDictionaryRatio, + compressionCodec, + compressionLevel); spillInfo.partitionLengths[currentPartition] = written; // Store the checksum for the current partition. @@ -578,7 +585,13 @@ public void writeSortedFileNative(boolean isLastFile) throws IOException { if (currentPartition != -1) { long written = doSpilling( - dataTypes, spillInfo.file, rowPartition, writeMetricsToUse, preferDictionaryRatio); + dataTypes, + spillInfo.file, + rowPartition, + writeMetricsToUse, + preferDictionaryRatio, + compressionCodec, + compressionLevel); spillInfo.partitionLengths[currentPartition] = written; synchronized (spills) { diff --git a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java index dcb9d99d37..006e8ce971 100644 --- a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java +++ b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/CometDiskBlockWriter.java @@ -103,6 +103,8 @@ public final class CometDiskBlockWriter { private long totalWritten = 0L; private boolean initialized = false; private final int columnarBatchSize; + private final String compressionCodec; + private final int compressionLevel; private final boolean isAsync; private final int asyncThreadNum; private final ExecutorService threadPool; @@ -153,6 +155,9 @@ public final class CometDiskBlockWriter { this.threadPool = threadPool; this.columnarBatchSize = (int) CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_BATCH_SIZE().get(); + this.compressionCodec = CometConf$.MODULE$.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC().get(); + this.compressionLevel = + (int) CometConf$.MODULE$.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL().get(); this.numElementsForSpillThreshold = (int) CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_SPILL_THRESHOLD().get(); @@ -397,7 +402,14 @@ long doSpilling(boolean isLast) throws IOException { synchronized (file) { outputRecords += rowPartition.getNumRows(); written = - doSpilling(dataTypes, file, rowPartition, writeMetricsToUse, preferDictionaryRatio); + doSpilling( + dataTypes, + file, + rowPartition, + writeMetricsToUse, + preferDictionaryRatio, + compressionCodec, + compressionLevel); } // Update metrics diff --git a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java index 3dc86b05bb..f3e814ab04 100644 --- a/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java +++ b/spark/src/main/java/org/apache/spark/sql/comet/execution/shuffle/SpillWriter.java @@ -171,7 +171,9 @@ protected long doSpilling( File file, RowPartition rowPartition, ShuffleWriteMetricsReporter writeMetricsToUse, - double preferDictionaryRatio) { + double preferDictionaryRatio, + String compressionCodec, + int compressionLevel) { long[] addresses = rowPartition.getRowAddresses(); int[] sizes = rowPartition.getRowSizes(); @@ -180,6 +182,7 @@ protected long doSpilling( long start = System.nanoTime(); int batchSize = (int) CometConf.COMET_COLUMNAR_SHUFFLE_BATCH_SIZE().get(); + boolean enableFastEncoding = (boolean) CometConf.COMET_SHUFFLE_ENABLE_FAST_ENCODING().get(); long[] results = nativeLib.writeSortedFileNative( addresses, @@ -190,7 +193,10 @@ protected long doSpilling( batchSize, checksumEnabled, checksumAlgo, - currentChecksum); + currentChecksum, + compressionCodec, + compressionLevel, + enableFastEncoding); long written = results[0]; checksum = results[1]; diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 08b24e0291..7a70181401 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -20,10 +20,11 @@ package org.apache.comet import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} import org.apache.comet.vector.NativeUtil /** @@ -52,7 +53,8 @@ class CometExecIterator( nativeMetrics: CometMetricNode, numParts: Int, partitionIndex: Int) - extends Iterator[ColumnarBatch] { + extends Iterator[ColumnarBatch] + with Logging { private val nativeLib = new Native() private val nativeUtil = new NativeUtil() @@ -73,8 +75,10 @@ class CometExecIterator( new CometTaskMemoryManager(id), batchSize = COMET_BATCH_SIZE.get(), use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false), + memory_pool_type = COMET_EXEC_MEMORY_POOL_TYPE.get(), memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf), - memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(), + memory_limit_per_task = getMemoryLimitPerTask(conf), + task_attempt_id = TaskContext.get().taskAttemptId, debug = COMET_DEBUG_ENABLED.get(), explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), workerThreads = COMET_WORKER_THREADS.get(), @@ -82,9 +86,37 @@ class CometExecIterator( } private var nextBatch: Option[ColumnarBatch] = None + private var prevBatch: ColumnarBatch = null private var currentBatch: ColumnarBatch = null private var closed: Boolean = false + private def getMemoryLimitPerTask(conf: SparkConf): Long = { + val numCores = numDriverOrExecutorCores(conf).toFloat + val maxMemory = CometSparkSessionExtensions.getCometMemoryOverhead(conf) + val coresPerTask = conf.get("spark.task.cpus", "1").toFloat + // example 16GB maxMemory * 16 cores with 4 cores per task results + // in memory_limit_per_task = 16 GB * 4 / 16 = 16 GB / 4 = 4GB + val limit = (maxMemory.toFloat * coresPerTask / numCores).toLong + logInfo( + s"Calculated per-task memory limit of $limit ($maxMemory * $coresPerTask / $numCores)") + limit + } + + private def numDriverOrExecutorCores(conf: SparkConf): Int = { + def convertToInt(threads: String): Int = { + if (threads == "*") Runtime.getRuntime.availableProcessors() else threads.toInt + } + val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r + val master = conf.get("spark.master") + master match { + case "local" => 1 + case LOCAL_N_REGEX(threads) => convertToInt(threads) + case LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) + case _ => conf.get("spark.executor.cores", "1").toInt + } + } + def getNextBatch(): Option[ColumnarBatch] = { assert(partitionIndex >= 0 && partitionIndex < numParts) @@ -103,6 +135,14 @@ class CometExecIterator( return true } + // Close previous batch if any. + // This is to guarantee safety at the native side before we overwrite the buffer memory + // shared across batches in the native side. + if (prevBatch != null) { + prevBatch.close() + prevBatch = null + } + nextBatch = getNextBatch() if (nextBatch.isEmpty) { @@ -125,6 +165,7 @@ class CometExecIterator( } currentBatch = nextBatch.get + prevBatch = currentBatch nextBatch = None currentBatch } diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 51e111f49b..c9d8ce55b1 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -596,7 +596,7 @@ class CometSparkSessionExtensions case op: SortMergeJoinExec if CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.get(conf) && - op.children.forall(isCometNative(_)) => + op.children.forall(isCometNative) => val newOp = transform1(op) newOp match { case Some(nativeOp) => @@ -817,68 +817,93 @@ class CometSparkSessionExtensions } // Native shuffle for Comet operators - case s: ShuffleExchangeExec - if isCometShuffleEnabled(conf) && - isCometNativeShuffleMode(conf) && - QueryPlanSerde.supportPartitioning(s.child.output, s.outputPartitioning)._1 => - logInfo("Comet extension enabled for Native Shuffle") + case s: ShuffleExchangeExec => + val nativePrecondition = isCometShuffleEnabled(conf) && + isCometNativeShuffleMode(conf) && + QueryPlanSerde.supportPartitioning(s.child.output, s.outputPartitioning)._1 - val newOp = transform1(s) - newOp match { - case Some(nativeOp) => - // Switch to use Decimal128 regardless of precision, since Arrow native execution - // doesn't support Decimal32 and Decimal64 yet. - conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") - val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) - CometSinkPlaceHolder(nativeOp, s, cometOp) - case None => - s - } + val nativeShuffle: Option[SparkPlan] = + if (nativePrecondition) { + val newOp = transform1(s) + newOp match { + case Some(nativeOp) => + // Switch to use Decimal128 regardless of precision, since Arrow native execution + // doesn't support Decimal32 and Decimal64 yet. + conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") + val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case None => + None + } + } else { + None + } - // Columnar shuffle for regular Spark operators (not Comet) and Comet operators - // (if configured). - // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not - // convert it to CometColumnarShuffle, - case s: ShuffleExchangeExec - if isCometShuffleEnabled(conf) && isCometJVMShuffleMode(conf) && + // this is a temporary workaround because some Spark SQL tests fail + // when we enable COMET_SHUFFLE_FALLBACK_TO_COLUMNAR due to valid bugs + // that we had not previously seen + val tryColumnarNext = + !nativePrecondition || (nativePrecondition && nativeShuffle.isEmpty && + COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.get(conf)) + + val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { + nativeShuffle + } else if (tryColumnarNext) { + // Columnar shuffle for regular Spark operators (not Comet) and Comet operators + // (if configured). + // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not + // convert it to CometColumnarShuffle, + if (isCometShuffleEnabled(conf) && isCometJVMShuffleMode(conf) && QueryPlanSerde.supportPartitioningTypes(s.child.output, s.outputPartitioning)._1 && - !isShuffleOperator(s.child) => - logInfo("Comet extension enabled for JVM Columnar Shuffle") + !isShuffleOperator(s.child)) { - val newOp = QueryPlanSerde.operator2Proto(s) - newOp match { - case Some(nativeOp) => - s.child match { - case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => - val cometOp = CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) - CometSinkPlaceHolder(nativeOp, s, cometOp) - case _ => - s + val newOp = QueryPlanSerde.operator2Proto(s) + newOp match { + case Some(nativeOp) => + s.child match { + case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => + val cometOp = + CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case _ => + None + } + case None => + None } - case None => - s + } else { + None + } + } else { + None } - case s: ShuffleExchangeExec => - val isShuffleEnabled = isCometShuffleEnabled(conf) - val outputPartitioning = s.outputPartitioning - val reason = getCometShuffleNotEnabledReason(conf).getOrElse("no reason available") - val msg1 = createMessage(!isShuffleEnabled, s"Comet shuffle is not enabled: $reason") - val columnarShuffleEnabled = isCometJVMShuffleMode(conf) - val msg2 = createMessage( - isShuffleEnabled && !columnarShuffleEnabled && !QueryPlanSerde - .supportPartitioning(s.child.output, outputPartitioning) - ._1, - "Native shuffle: " + - s"${QueryPlanSerde.supportPartitioning(s.child.output, outputPartitioning)._2}") - val msg3 = createMessage( - isShuffleEnabled && columnarShuffleEnabled && !QueryPlanSerde + if (nativeOrColumnarShuffle.isDefined) { + nativeOrColumnarShuffle.get + } else { + val isShuffleEnabled = isCometShuffleEnabled(conf) + val outputPartitioning = s.outputPartitioning + val reason = getCometShuffleNotEnabledReason(conf).getOrElse("no reason available") + val msg1 = createMessage(!isShuffleEnabled, s"Comet shuffle is not enabled: $reason") + val columnarShuffleEnabled = isCometJVMShuffleMode(conf) + val msg2 = createMessage( + isShuffleEnabled && !columnarShuffleEnabled && !QueryPlanSerde + .supportPartitioning(s.child.output, outputPartitioning) + ._1, + "Native shuffle: " + + s"${QueryPlanSerde.supportPartitioning(s.child.output, outputPartitioning)._2}") + val typeInfo = QueryPlanSerde .supportPartitioningTypes(s.child.output, outputPartitioning) - ._1, - "JVM shuffle: " + - s"${QueryPlanSerde.supportPartitioningTypes(s.child.output, outputPartitioning)._2}") - withInfo(s, Seq(msg1, msg2, msg3).flatten.mkString(",")) - s + ._2 + val msg3 = createMessage( + isShuffleEnabled && columnarShuffleEnabled && !QueryPlanSerde + .supportPartitioningTypes(s.child.output, outputPartitioning) + ._1, + "JVM shuffle: " + + s"$typeInfo") + withInfo(s, Seq(msg1, msg2, msg3).flatten.mkString(",")) + s + } case op => op match { @@ -995,8 +1020,7 @@ class CometSparkSessionExtensions if (CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.get()) { val info = new ExtendedExplainInfo() if (info.extensionInfo(newPlan).nonEmpty) { - // scalastyle:off println - println( + logWarning( "Comet cannot execute some parts of this plan natively " + s"(set ${CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key}=false " + "to disable this logging):\n" + @@ -1048,20 +1072,12 @@ class CometSparkSessionExtensions var firstNativeOp = true newPlan.transformDown { case op: CometNativeExec => - val newPlan = if (firstNativeOp) { + if (firstNativeOp) { firstNativeOp = false op.convertBlock() } else { op } - - // If reaching leaf node, reset `firstNativeOp` to true - // because it will start a new block in next iteration. - if (op.children.isEmpty) { - firstNativeOp = true - } - - newPlan case op => firstNativeOp = true op @@ -1123,6 +1139,17 @@ class CometSparkSessionExtensions override def apply(plan: SparkPlan): SparkPlan = { val eliminatedPlan = plan transformUp { case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => sparkToColumnar.child + case c @ ColumnarToRowExec(child) if child.exists(_.isInstanceOf[CometPlan]) => + val op = CometColumnarToRowExec(child) + if (c.logicalLink.isEmpty) { + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_TAG) + op.unsetTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG) + } else { + c.logicalLink.foreach(op.setLogicalLink) + } + op + case CometColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => + sparkToColumnar.child case CometSparkToColumnarExec(child: CometSparkToColumnarExec) => child // Spark adds `RowToColumnar` under Comet columnar shuffle. But it's redundant as the // shuffle takes row-based input. @@ -1139,6 +1166,8 @@ class CometSparkSessionExtensions eliminatedPlan match { case ColumnarToRowExec(child: CometCollectLimitExec) => child + case CometColumnarToRowExec(child: CometCollectLimitExec) => + child case other => other } diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 82c0373f4f..c293905c96 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -19,6 +19,8 @@ package org.apache.comet +import java.nio.ByteBuffer + import org.apache.spark.CometTaskMemoryManager import org.apache.spark.sql.comet.CometMetricNode @@ -54,8 +56,10 @@ class Native extends NativeBase { taskMemoryManager: CometTaskMemoryManager, batchSize: Int, use_unified_memory_manager: Boolean, + memory_pool_type: String, memory_limit: Long, - memory_fraction: Double, + memory_limit_per_task: Long, + task_attempt_id: Long, debug: Boolean, explain: Boolean, workerThreads: Int, @@ -118,9 +122,14 @@ class Native extends NativeBase { * @param currentChecksum * the current checksum of the file. As the checksum is computed incrementally, this is used * to resume the computation of checksum for previous written data. + * @param compressionCodec + * the compression codec + * @param compressionLevel + * the compression level * @return * [the number of bytes written to disk, the checksum] */ + // scalastyle:off @native def writeSortedFileNative( addresses: Array[Long], rowSizes: Array[Int], @@ -130,7 +139,11 @@ class Native extends NativeBase { batchSize: Int, checksumEnabled: Boolean, checksumAlgo: Int, - currentChecksum: Long): Array[Long] + currentChecksum: Long, + compressionCodec: String, + compressionLevel: Int, + enableFastEncoding: Boolean): Array[Long] + // scalastyle:on /** * Sorts partition ids of Spark unsafe rows in place. Used by Comet shuffle external sorter. @@ -141,4 +154,22 @@ class Native extends NativeBase { * the size of the array. */ @native def sortRowPartitionsNative(addr: Long, size: Long): Unit + + /** + * Decompress and decode a native shuffle block. + * @param shuffleBlock + * the encoded anc compressed shuffle block. + * @param length + * the limit of the byte buffer. + * @param addr + * the address of the array of compressed and encoded bytes. + * @param size + * the size of the array. + */ + @native def decodeShuffleBlock( + shuffleBlock: ByteBuffer, + length: Int, + arrayAddrs: Array[Long], + schemaAddrs: Array[Long]): Long + } diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 859cb13bea..6b0b10d808 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -204,7 +204,6 @@ object CometCast { Compatible() case DataTypes.StringType => Compatible() case DataTypes.DateType => Compatible() - case _: DecimalType => Compatible() case _ => Unsupported } } diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 174c1f6a29..244d24385e 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -373,6 +373,13 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim inputs: Seq[Attribute], binding: Boolean, conf: SQLConf): Option[AggExpr] = { + + if (aggExpr.isDistinct) { + // https://github.com/apache/datafusion-comet/issues/1260 + withInfo(aggExpr, "distinct aggregates are not supported") + return None + } + aggExpr.aggregateFunction match { case s @ Sum(child, _) if sumDataTypeSupported(s.dataType) && isLegacyMode(s) => val childExpr = exprToProto(child, inputs, binding) @@ -1747,7 +1754,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim exprToProtoInternal(elements._1, inputs) }) val thenSeq = branches.map(elements => { - allBranches = allBranches :+ elements._1 + allBranches = allBranches :+ elements._2 exprToProtoInternal(elements._2, inputs) }) assert(whenSeq.length == thenSeq.length) @@ -1790,10 +1797,19 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim optExprWithInfo(optExpr, expr, child) case InitCap(child) => - val castExpr = Cast(child, StringType) - val childExpr = exprToProtoInternal(castExpr, inputs) - val optExpr = scalarExprToProto("initcap", childExpr) - optExprWithInfo(optExpr, expr, castExpr) + if (CometConf.COMET_EXEC_INITCAP_ENABLED.get()) { + val castExpr = Cast(child, StringType) + val childExpr = exprToProtoInternal(castExpr, inputs) + val optExpr = scalarExprToProto("initcap", childExpr) + optExprWithInfo(optExpr, expr, castExpr) + } else { + withInfo( + expr, + "Comet initCap is not compatible with Spark yet. " + + "See https://github.com/apache/datafusion-comet/issues/1052 ." + + s"Set ${CometConf.COMET_EXEC_INITCAP_ENABLED.key}=true to enable it anyway.") + None + } case Length(child) => val castExpr = Cast(child, StringType) @@ -2268,6 +2284,18 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim withInfo(expr, "unsupported arguments for GetArrayStructFields", child) None } + case expr if expr.prettyName == "array_remove" => + createBinaryExpr( + expr.children(0), + expr.children(1), + inputs, + (builder, binaryExpr) => builder.setArrayRemove(binaryExpr)) + case expr if expr.prettyName == "array_contains" => + createBinaryExpr( + expr.children(0), + expr.children(1), + inputs, + (builder, binaryExpr) => builder.setArrayContains(binaryExpr)) case _ if expr.prettyName == "array_append" => createBinaryExpr( expr.children(0), @@ -2932,11 +2960,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim case RightOuter => JoinType.RightOuter case FullOuter => JoinType.FullOuter case LeftSemi => JoinType.LeftSemi - // TODO: DF SMJ with join condition fails TPCH q21 - case LeftAnti if condition.isEmpty => JoinType.LeftAnti - case LeftAnti => - withInfo(join, "LeftAnti SMJ join with condition is not supported") - return None + case LeftAnti => JoinType.LeftAnti case _ => // Spark doesn't support other join types withInfo(op, s"Unsupported join type ${join.joinType}") @@ -3225,17 +3249,22 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim orderSpec: Seq[SortOrder], op: SparkPlan): Boolean = { if (partitionSpec.length != orderSpec.length) { - withInfo(op, "Partitioning and sorting specifications do not match") return false } - val partitionColumnNames = partitionSpec.collect { case a: AttributeReference => - a.name + val partitionColumnNames = partitionSpec.collect { + case a: AttributeReference => a.name + case other => + withInfo(op, s"Unsupported partition expression: ${other.getClass.getSimpleName}") + return false } val orderColumnNames = orderSpec.collect { case s: SortOrder => s.child match { case a: AttributeReference => a.name + case other => + withInfo(op, s"Unsupported sort expression: ${other.getClass.getSimpleName}") + return false } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index ccf218cf6c..6bc519ab95 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.comet +import java.io.DataInputStream +import java.nio.channels.Channels import java.util.UUID import java.util.concurrent.{Future, TimeoutException, TimeUnit} @@ -26,13 +28,15 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.concurrent.duration.NANOSECONDS import scala.util.control.NonFatal -import org.apache.spark.{broadcast, Partition, SparkContext, TaskContext} +import org.apache.spark.{broadcast, Partition, SparkContext, SparkEnv, TaskContext} import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec +import org.apache.spark.io.CompressionCodec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.comet.execution.shuffle.ArrowReaderIterator import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec} @@ -299,7 +303,23 @@ class CometBatchRDD( override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { val partition = split.asInstanceOf[CometBatchPartition] partition.value.value.toIterator - .flatMap(CometExec.decodeBatches(_, this.getClass.getSimpleName)) + .flatMap(decodeBatches(_, this.getClass.getSimpleName)) + } + + /** + * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. + */ + private def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { + if (bytes.size == 0) { + return Iterator.empty + } + + // use Spark's compression codec (LZ4 by default) and not Comet's compression + val codec = CompressionCodec.createCodec(SparkEnv.get.conf) + val cbbis = bytes.toInputStream() + val ins = new DataInputStream(codec.compressedInputStream(cbbis)) + // batches are in Arrow IPC format + new ArrowReaderIterator(Channels.newChannel(ins), source) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala new file mode 100644 index 0000000000..18d95a473c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala @@ -0,0 +1,209 @@ +/* + * 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.comet + +import scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{CodegenSupport, ColumnarToRowTransition, SparkPlan} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, WritableColumnVector} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.util.Utils + +import org.apache.comet.vector.CometPlainVector + +/** + * Copied from Spark `ColumnarToRowExec`. Comet needs the fix for SPARK-50235 but cannot wait for + * the fix to be released in Spark versions. We copy the implementation here to apply the fix. + */ +case class CometColumnarToRowExec(child: SparkPlan) + extends ColumnarToRowTransition + with CodegenSupport { + // supportsColumnar requires to be only called on driver side, see also SPARK-37779. + assert(Utils.isInRunningSparkTask || child.supportsColumnar) + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + // `ColumnarToRowExec` processes the input RDD directly, which is kind of a leaf node in the + // codegen stage and needs to do the limit check. + protected override def canCheckLimitNotReached: Boolean = true + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches")) + + override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + // This avoids calling `output` in the RDD closure, so that we don't need to include the entire + // plan (this) in the closure. + val localOutput = this.output + child.executeColumnar().mapPartitionsInternal { batches => + val toUnsafe = UnsafeProjection.create(localOutput, localOutput) + batches.flatMap { batch => + numInputBatches += 1 + numOutputRows += batch.numRows() + batch.rowIterator().asScala.map(toUnsafe) + } + } + } + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. This is called once + * per [[ColumnVector]] in the batch. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = CodeGenerator.javaType(dataType) + val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { + JavaCode.isNullVariable(ctx.freshName("isNull")) + } else { + FalseLiteral + } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = code"${ctx.registerComment(str)}" + (if (nullable) { + code""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${CodeGenerator.defaultValue(dataType)} : ($value); + """ + } else { + code"$javaType $valueVar = $value;" + }) + ExprCode(code, isNullVar, JavaCode.variable(valueVar, dataType)) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. This produces an + * [[org.apache.spark.sql.catalyst.expressions.UnsafeRow]] for each row in each batch. + */ + override protected def doProduce(ctx: CodegenContext): String = { + // PhysicalRDD always just has one input + val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val numInputBatches = metricTerm(ctx, "numInputBatches") + + val columnarBatchClz = classOf[ColumnarBatch].getName + val batch = ctx.addMutableState(columnarBatchClz, "batch") + + val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init as batchIdx = 0 + val columnVectorClzs = + child.vectorTypes.getOrElse(Seq.fill(output.indices.size)(classOf[ColumnVector].getName)) + val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map { + case (columnVectorClz, i) => + val name = ctx.addMutableState(columnVectorClz, s"colInstance$i") + (name, s"$name = ($columnVectorClz) $batch.column($i);") + }.unzip + + val nextBatch = ctx.freshName("nextBatch") + val nextBatchFuncName = ctx.addNewFunction( + nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $numInputBatches.add(1); + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + val localIdx = ctx.freshName("localIdx") + val localEnd = ctx.freshName("localEnd") + val numRows = ctx.freshName("numRows") + val shouldStop = if (parent.needStopCheck) { + s"if (shouldStop()) { $idx = $rowidx + 1; return; }" + } else { + "// shouldStop check is eliminated" + } + + val writableColumnVectorClz = classOf[WritableColumnVector].getName + val constantColumnVectorClz = classOf[ConstantColumnVector].getName + val cometPlainColumnVectorClz = classOf[CometPlainVector].getName + + // scalastyle:off line.size.limit + s""" + |if ($batch == null) { + | $nextBatchFuncName(); + |} + |while ($limitNotReachedCond $batch != null) { + | int $numRows = $batch.numRows(); + | int $localEnd = $numRows - $idx; + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | int $rowidx = $idx + $localIdx; + | ${consume(ctx, columnsBatchInput).trim} + | $shouldStop + | } + | $idx = $numRows; + | + | // Comet fix for SPARK-50235 + | for (int i = 0; i < ${colVars.length}; i++) { + | if (!($batch.column(i) instanceof $writableColumnVectorClz || $batch.column(i) instanceof $constantColumnVectorClz || $batch.column(i) instanceof $cometPlainColumnVectorClz)) { + | $batch.column(i).close(); + | } else if ($batch.column(i) instanceof $cometPlainColumnVectorClz) { + | $cometPlainColumnVectorClz cometPlainColumnVector = ($cometPlainColumnVectorClz) $batch.column(i); + | if (!cometPlainColumnVector.isReused()) { + | cometPlainColumnVector.close(); + | } + | } + | } + | + | $batch = null; + | $nextBatchFuncName(); + |} + |// Comet fix for SPARK-50235: clean up resources + |if ($batch != null) { + | $batch.close(); + |} + """.stripMargin + // scalastyle:on line.size.limit + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + Seq(child.executeColumnar().asInstanceOf[RDD[InternalRow]]) // Hack because of type erasure + } + + override protected def withNewChildInternal(newChild: SparkPlan): CometColumnarToRowExec = + copy(child = newChild) +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala index a26fa28c8b..53370a03b7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -132,10 +132,11 @@ object CometMetricNode { def shuffleMetrics(sc: SparkContext): Map[String, SQLMetric] = { Map( - "elapsed_compute" -> SQLMetrics.createNanoTimingMetric(sc, "native shuffle time"), + "elapsed_compute" -> SQLMetrics.createNanoTimingMetric(sc, "native shuffle writer time"), "mempool_time" -> SQLMetrics.createNanoTimingMetric(sc, "memory pool time"), "repart_time" -> SQLMetrics.createNanoTimingMetric(sc, "repartition time"), - "ipc_time" -> SQLMetrics.createNanoTimingMetric(sc, "encoding and compression time"), + "encode_time" -> SQLMetrics.createNanoTimingMetric(sc, "encoding and compression time"), + "decode_time" -> SQLMetrics.createNanoTimingMetric(sc, "decoding and decompression time"), "spill_count" -> SQLMetrics.createMetric(sc, "number of spills"), "spilled_bytes" -> SQLMetrics.createMetric(sc, "spilled bytes"), "input_batches" -> SQLMetrics.createMetric(sc, "number of input batches")) diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ArrowReaderIterator.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala similarity index 89% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala index e026cbeb1a..1283a745a6 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometBlockStoreShuffleReader.scala @@ -21,17 +21,14 @@ package org.apache.spark.sql.comet.execution.shuffle import java.io.InputStream -import org.apache.spark.InterruptibleIterator -import org.apache.spark.MapOutputTracker -import org.apache.spark.SparkEnv -import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config +import org.apache.spark.{InterruptibleIterator, MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.BaseShuffleHandle import org.apache.spark.shuffle.ShuffleReader import org.apache.spark.shuffle.ShuffleReadMetricsReporter +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.BlockId import org.apache.spark.storage.BlockManager import org.apache.spark.storage.BlockManagerId @@ -88,7 +85,7 @@ class CometBlockStoreShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - var currentReadIterator: ArrowReaderIterator = null + var currentReadIterator: NativeBatchDecoderIterator = null // Closes last read iterator after the task is finished. // We need to close read iterator during iterating input streams, @@ -100,18 +97,16 @@ class CometBlockStoreShuffleReader[K, C]( } } - val recordIter = fetchIterator - .flatMap { case (_, inputStream) => - IpcInputStreamIterator(inputStream, decompressingNeeded = true, context) - .flatMap { channel => - if (currentReadIterator != null) { - // Closes previous read iterator. - currentReadIterator.close() - } - currentReadIterator = new ArrowReaderIterator(channel, this.getClass.getSimpleName) - currentReadIterator.map((0, _)) // use 0 as key since it's not used - } - } + val recordIter: Iterator[(Int, ColumnarBatch)] = fetchIterator + .flatMap(blockIdAndStream => { + if (currentReadIterator != null) { + currentReadIterator.close() + } + currentReadIterator = + NativeBatchDecoderIterator(blockIdAndStream._2, context, dep.decodeTime) + currentReadIterator + }) + .map(b => (0, b)) // Update the context task metrics for each record read. val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala similarity index 94% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala index 7b1d1f1271..8c8aed28ee 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala @@ -25,6 +25,7 @@ import org.apache.spark.{Aggregator, Partitioner, ShuffleDependency, SparkEnv} import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleWriteProcessor +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType /** @@ -39,7 +40,8 @@ class CometShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( override val mapSideCombine: Boolean = false, override val shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor, val shuffleType: ShuffleType = CometNativeShuffle, - val schema: Option[StructType] = None) + val schema: Option[StructType] = None, + val decodeTime: SQLMetric) extends ShuffleDependency[K, V, C]( _rdd, partitioner, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 0cd8a9ce67..7c27fb1969 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -52,8 +52,9 @@ import org.apache.spark.util.random.XORShiftRandom import com.google.common.base.Objects +import org.apache.comet.CometConf import org.apache.comet.serde.{OperatorOuterClass, PartitioningOuterClass, QueryPlanSerde} -import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator} import org.apache.comet.serde.QueryPlanSerde.serializeDataType import org.apache.comet.shims.ShimCometShuffleExchangeExec @@ -237,7 +238,8 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { partitioner = new Partitioner { override def numPartitions: Int = outputPartitioning.numPartitions override def getPartition(key: Any): Int = key.asInstanceOf[Int] - }) + }, + decodeTime = metrics("decode_time")) dependency } @@ -434,7 +436,8 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics), shuffleType = CometColumnarShuffle, - schema = Some(fromAttributes(outputAttributes))) + schema = Some(fromAttributes(outputAttributes)), + decodeTime = writeMetrics("decode_time")) dependency } @@ -480,7 +483,7 @@ class CometShuffleWriteProcessor( val detailedMetrics = Seq( "elapsed_compute", - "ipc_time", + "encode_time", "repart_time", "mempool_time", "input_batches", @@ -552,6 +555,22 @@ class CometShuffleWriteProcessor( val shuffleWriterBuilder = OperatorOuterClass.ShuffleWriter.newBuilder() shuffleWriterBuilder.setOutputDataFile(dataFile) shuffleWriterBuilder.setOutputIndexFile(indexFile) + shuffleWriterBuilder.setEnableFastEncoding( + CometConf.COMET_SHUFFLE_ENABLE_FAST_ENCODING.get()) + + if (SparkEnv.get.conf.getBoolean("spark.shuffle.compress", true)) { + val codec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() match { + case "zstd" => CompressionCodec.Zstd + case "lz4" => CompressionCodec.Lz4 + case "snappy" => CompressionCodec.Snappy + case other => throw new UnsupportedOperationException(s"invalid codec: $other") + } + shuffleWriterBuilder.setCodec(codec) + } else { + shuffleWriterBuilder.setCodec(CompressionCodec.None) + } + shuffleWriterBuilder.setCompressionLevel( + CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) outputPartitioning match { case _: HashPartitioning => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala index ef67167c4b..b2cc2c2bad 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala @@ -243,7 +243,7 @@ object CometShuffleManager extends Logging { lazy val compressionCodecForShuffling: CompressionCodec = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) + val codecName = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get(SQLConf.get) // only zstd compression is supported at the moment if (codecName != "zstd") { diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala similarity index 95% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index af78ed2905..ba6fc588e2 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.comet.execution.shuffle -import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.execution.{CoalescedMapperPartitionSpec, CoalescedPartitioner, CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala new file mode 100644 index 0000000000..b8c1669494 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala @@ -0,0 +1,190 @@ +/* + * 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.comet.execution.shuffle + +import java.io.{EOFException, InputStream} +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.channels.{Channels, ReadableByteChannel} + +import org.apache.spark.TaskContext +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.Native +import org.apache.comet.vector.NativeUtil + +/** + * This iterator wraps a Spark input stream that is reading shuffle blocks generated by the Comet + * native ShuffleWriterExec and then calls native code to decompress and decode the shuffle blocks + * and use Arrow FFI to return the Arrow record batch. + */ +case class NativeBatchDecoderIterator( + var in: InputStream, + taskContext: TaskContext, + decodeTime: SQLMetric) + extends Iterator[ColumnarBatch] { + + private var isClosed = false + private val longBuf = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN) + private val native = new Native() + private val nativeUtil = new NativeUtil() + private var currentBatch: ColumnarBatch = null + private var batch = fetchNext() + + import NativeBatchDecoderIterator.threadLocalDataBuf + + if (taskContext != null) { + taskContext.addTaskCompletionListener[Unit](_ => { + close() + }) + } + + private val channel: ReadableByteChannel = if (in != null) { + Channels.newChannel(in) + } else { + null + } + + def hasNext(): Boolean = { + if (channel == null || isClosed) { + return false + } + if (batch.isDefined) { + return true + } + + // Release the previous batch. + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + + batch = fetchNext() + if (batch.isEmpty) { + close() + return false + } + true + } + + def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException + } + + val nextBatch = batch.get + + currentBatch = nextBatch + batch = None + currentBatch + } + + private def fetchNext(): Option[ColumnarBatch] = { + if (channel == null || isClosed) { + return None + } + + // read compressed batch size from header + try { + longBuf.clear() + while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} + } catch { + case _: EOFException => + close() + return None + } + + // If we reach the end of the stream, we are done, or if we read partial length + // then the stream is corrupted. + if (longBuf.hasRemaining) { + if (longBuf.position() == 0) { + close() + return None + } + throw new EOFException("Data corrupt: unexpected EOF while reading compressed ipc lengths") + } + + // get compressed length (including headers) + longBuf.flip() + val compressedLength = longBuf.getLong + + // read field count from header + longBuf.clear() + while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} + if (longBuf.hasRemaining) { + throw new EOFException("Data corrupt: unexpected EOF while reading field count") + } + longBuf.flip() + val fieldCount = longBuf.getLong.toInt + + // read body + val bytesToRead = compressedLength - 8 + if (bytesToRead > Integer.MAX_VALUE) { + // very unlikely that shuffle block will reach 2GB + throw new IllegalStateException( + s"Native shuffle block size of $bytesToRead exceeds " + + s"maximum of ${Integer.MAX_VALUE}. Try reducing shuffle batch size.") + } + var dataBuf = threadLocalDataBuf.get() + if (dataBuf.capacity() < bytesToRead) { + // it is unlikely that we would overflow here since it would + // require a 1GB compressed shuffle block but we check anyway + val newCapacity = (bytesToRead * 2L).min(Integer.MAX_VALUE).toInt + dataBuf = ByteBuffer.allocateDirect(newCapacity) + threadLocalDataBuf.set(dataBuf) + } + dataBuf.clear() + dataBuf.limit(bytesToRead.toInt) + while (dataBuf.hasRemaining && channel.read(dataBuf) >= 0) {} + if (dataBuf.hasRemaining) { + throw new EOFException("Data corrupt: unexpected EOF while reading compressed batch") + } + + // make native call to decode batch + val startTime = System.nanoTime() + val batch = nativeUtil.getNextBatch( + fieldCount, + (arrayAddrs, schemaAddrs) => { + native.decodeShuffleBlock(dataBuf, bytesToRead.toInt, arrayAddrs, schemaAddrs) + }) + decodeTime.add(System.nanoTime() - startTime) + + batch + } + + def close(): Unit = { + synchronized { + if (!isClosed) { + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + in.close() + isClosed = true + } + } + } +} + +object NativeBatchDecoderIterator { + private val threadLocalDataBuf: ThreadLocal[ByteBuffer] = ThreadLocal.withInitial(() => { + ByteBuffer.allocateDirect(128 * 1024) + }) +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala similarity index 52% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala index eea134ab5e..23b4a5ec23 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/ShuffleUtils.scala @@ -21,22 +21,33 @@ package org.apache.spark.sql.comet.execution.shuffle import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.IO_COMPRESSION_CODEC +import org.apache.spark.internal.config.{IO_COMPRESSION_CODEC, SHUFFLE_COMPRESS} import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf private[spark] object ShuffleUtils extends Logging { - lazy val compressionCodecForShuffling: CompressionCodec = { + // optional compression codec to use when compressing shuffle files + lazy val compressionCodecForShuffling: Option[CompressionCodec] = { val sparkConf = SparkEnv.get.conf - val codecName = CometConf.COMET_EXEC_SHUFFLE_CODEC.get(SQLConf.get) - - // only zstd compression is supported at the moment - if (codecName != "zstd") { - logWarning( - s"Overriding config ${IO_COMPRESSION_CODEC}=${codecName} in shuffling, force using zstd") + val shuffleCompressionEnabled = sparkConf.getBoolean(SHUFFLE_COMPRESS.key, true) + val sparkShuffleCodec = sparkConf.get(IO_COMPRESSION_CODEC.key, "lz4") + val cometShuffleCodec = CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_CODEC.get() + if (shuffleCompressionEnabled) { + if (sparkShuffleCodec != cometShuffleCodec) { + logWarning( + s"Overriding config $IO_COMPRESSION_CODEC=$sparkShuffleCodec in shuffling, " + + s"force using $cometShuffleCodec") + } + cometShuffleCodec match { + case "zstd" => + Some(CompressionCodec.createCodec(sparkConf, "zstd")) + case other => + throw new UnsupportedOperationException( + s"Unsupported shuffle compression codec: $other") + } + } else { + None } - CompressionCodec.createCodec(sparkConf, "zstd") } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index f6e1860fde..9e2ca987fa 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.comet -import java.io.{ByteArrayOutputStream, DataInputStream} -import java.nio.channels.Channels +import java.io.ByteArrayOutputStream import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.io.CompressionCodec +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, NamedExpression, SortOrder} @@ -34,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning} -import org.apache.spark.sql.comet.execution.shuffle.{ArrowReaderIterator, CometShuffleExchangeExec} +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.comet.plans.PartitioningPreservingUnaryExecNode import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution.{BinaryExecNode, ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} @@ -78,18 +76,6 @@ abstract class CometExec extends CometPlan { // outputPartitioning of SparkPlan, e.g., AQEShuffleReadExec. override def outputPartitioning: Partitioning = originalPlan.outputPartitioning - /** - * Executes the Comet operator and returns the result as an iterator of ColumnarBatch. - */ - def executeColumnarCollectIterator(): (Long, Iterator[ColumnarBatch]) = { - val countsAndBytes = CometExec.getByteArrayRdd(this).collect() - val total = countsAndBytes.map(_._1).sum - val rows = countsAndBytes.iterator - .flatMap(countAndBytes => - CometExec.decodeBatches(countAndBytes._2, this.getClass.getSimpleName)) - (total, rows) - } - protected def setSubqueries(planId: Long, sparkPlan: SparkPlan): Unit = { sparkPlan.children.foreach(setSubqueries(planId, _)) @@ -161,21 +147,6 @@ object CometExec { Utils.serializeBatches(iter) } } - - /** - * Decodes the byte arrays back to ColumnarBatchs and put them into buffer. - */ - def decodeBatches(bytes: ChunkedByteBuffer, source: String): Iterator[ColumnarBatch] = { - if (bytes.size == 0) { - return Iterator.empty - } - - val codec = CompressionCodec.createCodec(SparkEnv.get.conf) - val cbbis = bytes.toInputStream() - val ins = new DataInputStream(codec.compressedInputStream(cbbis)) - - new ArrowReaderIterator(Channels.newChannel(ins), source) - } } /** 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 ccca281513..0844332b3b 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometTakeOrderedAndProject (39) +- CometProject (38) +- CometBroadcastHashJoin (37) @@ -224,14 +224,14 @@ Arguments: [c_customer_id#27], [c_customer_id#27] 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] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [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 (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (45) BroadcastExchange 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 5aa8750a76..3ca476a1f8 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id] CometProject [c_customer_id] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 e32e86f270..3c2df96933 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,50 @@ == 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 (46) ++- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * CometColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) CometScan parquet spark_catalog.default.customer @@ -101,7 +102,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +124,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +156,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +192,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +220,7 @@ ReadSchema: struct -(47) CometFilter +(48) 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 +(49) 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] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) 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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt index efd4b187dd..bc0ed49101 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,72 @@ 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + 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) + CometColumnarToRow 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 #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 609ab39c86..a01c703239 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (69) +* CometColumnarToRow (69) +- CometTakeOrderedAndProject (68) +- CometProject (67) +- CometBroadcastHashJoin (66) @@ -393,14 +393,14 @@ Arguments: [customer_preferred_cust_flag#36], [customer_preferred_cust_flag#36] 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] +(69) CometColumnarToRow [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 (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -416,7 +416,7 @@ ReadSchema: struct 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)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (73) BroadcastExchange @@ -425,7 +425,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometFilter (75) +- CometScan parquet spark_catalog.default.date_dim (74) @@ -441,7 +441,7 @@ ReadSchema: struct 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)) -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (77) BroadcastExchange 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 e509c8c3c1..8896bc14be 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_preferred_cust_flag] CometProject [customer_preferred_cust_flag] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,7 +47,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt index 18abad247b..7b8864d54f 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 e81a507139..c4c49fe339 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 e3c357b407..774d652f36 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometHashAggregate (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -188,14 +188,14 @@ 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))] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (38) -+- * ColumnarToRow (37) ++- * CometColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan parquet spark_catalog.default.date_dim (34) @@ -216,7 +216,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (38) BroadcastExchange 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 46ed9b6365..c4d3b744d0 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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))] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 bd6aa9273c..7d4428d5b2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometTakeOrderedAndProject (101) +- CometHashAggregate (100) +- CometExchange (99) @@ -569,13 +569,13 @@ Functions [2]: [sum(sales#49), sum(number_sales#50)] 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] +(102) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* ColumnarToRow (119) +* CometColumnarToRow (119) +- CometHashAggregate (118) +- CometExchange (117) +- CometHashAggregate (116) @@ -670,7 +670,7 @@ Input [2]: [sum#122, count#123] Keys: [] Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#124] Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * ColumnarToRow (123) ++- * CometColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) ColumnarToRow [codegen id : 1] +(123) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 < Input [2]: [d_date_sk#26, d_year#125] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (129) BroadcastExchange 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 852af8ad82..ec0b46d611 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -11,7 +11,7 @@ WholeStageCodegen (1) CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 @@ -45,7 +45,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -69,7 +69,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 4126fed5c3..a1c22982cb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_c 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] +(85) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#98] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#104] 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 bb4be4d3ae..befc6dadbb 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,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] 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 037c526267..3ee326ad53 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (23) +* CometColumnarToRow (23) +- CometTakeOrderedAndProject (22) +- CometHashAggregate (21) +- CometExchange (20) @@ -130,14 +130,14 @@ Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] 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] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -158,7 +158,7 @@ Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2) 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] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (28) BroadcastExchange 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 d417f1c682..425f20e10c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 acd12b2771..dc963d61f6 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * ColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.call_center (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * CometColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.call_center (29) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -203,7 +204,7 @@ 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))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -220,11 +221,14 @@ Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] Results [3]: [sum#20, sum#21, count#25] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#20, sum#21, count#25] + +(41) HashAggregate [codegen id : 2] Input [3]: [sum#20, sum#21, 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 f054ee03e9..daecc86311 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - 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] - CometExchange [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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [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] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] + CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] + CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [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_order_number,cs_warehouse_sk] + 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] + CometExchange [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] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [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] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + 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 496ec2f1a1..08c00a0bc4 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_qu 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] +(42) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] (52) BroadcastExchange 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 a3adaf1724..97000b91df 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] 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 c41a1ef7d4..2d08a0303c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometTakeOrderedAndProject (40) +- CometHashAggregate (39) +- CometExchange (38) @@ -232,14 +232,14 @@ Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_pric 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] +(41) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (46) BroadcastExchange 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 e26491c76a..eaf02960fe 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)))] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 b91551e576..c1bf0f14a0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -193,6 +193,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] 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] +(34) CometColumnarToRow [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 46b0d650a5..bfbf7fe94a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 d51ae75262..30e7e4ae49 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometSort (32) +- CometColumnarExchange (31) +- CometProject (30) @@ -183,6 +183,6 @@ Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREM 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] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [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 ed772bade0..38cb758b73 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 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 909acbf7f8..6a32155103 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 9a906085b8..147d91b586 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 31a5bef868..0340ef91c8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometFilter (22) +- CometHashAggregate (21) @@ -135,14 +135,14 @@ Condition : (CASE WHEN (inv_before#15 > 0) THEN (knownfloatingpointnormalized(no 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) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct 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] +(27) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange 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 6e483f6049..6ff97b53ac 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 526dadb3c8..759689b1b8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometHashAggregate (22) +- CometExchange (21) @@ -135,14 +135,14 @@ Functions [1]: [avg(inv_quantity_on_hand#3)] 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] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (29) BroadcastExchange 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 ed997b5d50..f9d4ce909d 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 0879422474..6a67b0f6d3 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (67) +* CometColumnarToRow (67) +- CometHashAggregate (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -370,14 +370,14 @@ Input [2]: [sum#48, isEmpty#49] Keys: [] Functions [1]: [sum(sales#32)] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [sum(sales)#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) 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] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -436,7 +436,7 @@ Input [2]: [d_date_sk#10, d_date#11] 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) +* CometColumnarToRow (95) +- CometHashAggregate (94) +- CometExchange (93) +- CometHashAggregate (92) @@ -536,12 +536,12 @@ Input [1]: [max#62] Keys: [] Functions [1]: [max(csales#61)] -(95) ColumnarToRow [codegen id : 1] +(95) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (100) -+- * ColumnarToRow (99) ++- * CometColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(99) ColumnarToRow [codegen id : 1] +(99) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#57] (100) BroadcastExchange 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 7606f105f3..4ecd0dfcc5 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -54,7 +54,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 b9d5b25fd4..8ed54d7afa 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (87) +* CometColumnarToRow (87) +- CometTakeOrderedAndProject (86) +- CometUnion (85) :- CometHashAggregate (62) @@ -479,14 +479,14 @@ Child 1 Input [3]: [c_last_name#52, c_first_name#51, sales#57] 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] +(87) CometColumnarToRow [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 (92) -+- * ColumnarToRow (91) ++- * CometColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometScan parquet spark_catalog.default.date_dim (88) @@ -507,7 +507,7 @@ Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2 Input [3]: [d_date_sk#32, d_year#33, d_moy#34] Arguments: [d_date_sk#32], [d_date_sk#32] -(91) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] (92) BroadcastExchange @@ -516,7 +516,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) -+- * ColumnarToRow (96) ++- * CometColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometScan parquet spark_catalog.default.date_dim (93) @@ -537,7 +537,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (97) BroadcastExchange @@ -545,7 +545,7 @@ Input [2]: [d_date_sk#10, d_date#11] 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) +* CometColumnarToRow (115) +- CometHashAggregate (114) +- CometExchange (113) +- CometHashAggregate (112) @@ -645,12 +645,12 @@ Input [1]: [max#69] Keys: [] Functions [1]: [max(csales#68)] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#70] Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#62 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -671,7 +671,7 @@ Condition : (d_year#65 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#64)) Input [2]: [d_date_sk#64, d_year#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (120) BroadcastExchange 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 d503bb68af..9493bffd25 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -40,7 +40,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -74,7 +74,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 9dbf4af839..0f0ae8e63d 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,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Condition : isnotnull(i_item_sk#51) -(54) CometBroadcastExchange +(56) CometBroadcastExchange Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] Right output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_item_sk#40], [i_item_sk#51], Inner, BuildRight -(56) CometProject +(58) CometProject Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56], [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Right output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight -(59) CometProject +(61) CometProject Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#61, ca_zip#62, ca_country#63] -(62) BroadcastHashJoin [codegen id : 2] +(64) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#60, s_zip#50] Right keys [2]: [upper(ca_country#63), ca_zip#62] Join type: Inner Join condition: None -(63) Project [codegen id : 2] +(65) Project [codegen id : 2] Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] -(64) HashAggregate [codegen id : 2] +(66) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum#64] Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -(65) Exchange +(67) CometColumnarExchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(68) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 3] +(69) HashAggregate [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#66] Keys: [] Functions [1]: [partial_avg(netpaid#66)] Aggregate Attributes [2]: [sum#67, count#68] Results [2]: [sum#69, count#70] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#69, count#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#69, count#70] Keys: [] Functions [1]: [avg(netpaid#66)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt index f27ae40199..c9756da952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Condition : isnotnull(i_item_sk#51) -(54) CometBroadcastExchange +(56) CometBroadcastExchange Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] Right output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_item_sk#40], [i_item_sk#51], Inner, BuildRight -(56) CometProject +(58) CometProject Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56], [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Right output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight -(59) CometProject +(61) CometProject Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#61, ca_zip#62, ca_country#63] -(62) BroadcastHashJoin [codegen id : 2] +(64) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#60, s_zip#50] Right keys [2]: [upper(ca_country#63), ca_zip#62] Join type: Inner Join condition: None -(63) Project [codegen id : 2] +(65) Project [codegen id : 2] Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] -(64) HashAggregate [codegen id : 2] +(66) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum#64] Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -(65) Exchange +(67) CometColumnarExchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(68) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 3] +(69) HashAggregate [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#66] Keys: [] Functions [1]: [partial_avg(netpaid#66)] Aggregate Attributes [2]: [sum#67, count#68] Results [2]: [sum#69, count#70] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#69, count#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#69, count#70] Keys: [] Functions [1]: [avg(netpaid#66)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt index 6c78335481..62c54c9f56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_lo Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#31 ASC NULLS FIRST,s_store_id#27 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#31,s_store_id#27,s_store_name#28,store_sales_profit#35,store_returns_loss#36,catalog_sales_profit#37]), [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37], 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) 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] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 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] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index 13cb05053c..e64e8bc840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt index b93df3398e..525313e3f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(Uns 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] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt index f643ff7b04..16f8d0848e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt index 4d1b12626a..28bc38292b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#20 ASC NULLS FIRST,s_state#21 ASC NULLS FIRST], output=[i_item_id#20,s_state#21,g_state#31,agg1#32,agg2#33,agg3#34,agg4#35]), [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35], 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt index 121f1be7ce..e65b02d4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,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))] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt index 8f0671c8f0..98f8e50c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt @@ -1,74 +1,80 @@ == 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) -: : : : : +- * ColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- * ColumnarToRow (16) -: : : : +- CometExchange (15) -: : : : +- CometHashAggregate (14) -: : : : +- CometProject (13) -: : : : +- CometFilter (12) -: : : : +- CometScan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- * ColumnarToRow (28) -: : : +- CometExchange (27) -: : : +- CometHashAggregate (26) -: : : +- CometProject (25) -: : : +- CometFilter (24) -: : : +- CometScan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- * ColumnarToRow (40) -: : +- CometExchange (39) -: : +- CometHashAggregate (38) -: : +- CometProject (37) -: : +- CometFilter (36) -: : +- CometScan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- * ColumnarToRow (52) -: +- CometExchange (51) -: +- CometHashAggregate (50) -: +- CometProject (49) -: +- CometFilter (48) -: +- CometScan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- * ColumnarToRow (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.store_sales (59) +* BroadcastNestedLoopJoin Inner BuildRight (76) +:- * BroadcastNestedLoopJoin Inner BuildRight (63) +: :- * BroadcastNestedLoopJoin Inner BuildRight (50) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) +: : : : :- * HashAggregate (11) +: : : : : +- * CometColumnarToRow (10) +: : : : : +- CometColumnarExchange (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * HashAggregate (7) +: : : : : +- * CometColumnarToRow (6) +: : : : : +- CometExchange (5) +: : : : : +- CometHashAggregate (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (23) +: : : : +- * HashAggregate (22) +: : : : +- * CometColumnarToRow (21) +: : : : +- CometColumnarExchange (20) +: : : : +- * HashAggregate (19) +: : : : +- * HashAggregate (18) +: : : : +- * CometColumnarToRow (17) +: : : : +- CometExchange (16) +: : : : +- CometHashAggregate (15) +: : : : +- CometProject (14) +: : : : +- CometFilter (13) +: : : : +- CometScan parquet spark_catalog.default.store_sales (12) +: : : +- BroadcastExchange (36) +: : : +- * HashAggregate (35) +: : : +- * CometColumnarToRow (34) +: : : +- CometColumnarExchange (33) +: : : +- * HashAggregate (32) +: : : +- * HashAggregate (31) +: : : +- * CometColumnarToRow (30) +: : : +- CometExchange (29) +: : : +- CometHashAggregate (28) +: : : +- CometProject (27) +: : : +- CometFilter (26) +: : : +- CometScan parquet spark_catalog.default.store_sales (25) +: : +- BroadcastExchange (49) +: : +- * HashAggregate (48) +: : +- * CometColumnarToRow (47) +: : +- CometColumnarExchange (46) +: : +- * HashAggregate (45) +: : +- * HashAggregate (44) +: : +- * CometColumnarToRow (43) +: : +- CometExchange (42) +: : +- CometHashAggregate (41) +: : +- CometProject (40) +: : +- CometFilter (39) +: : +- CometScan parquet spark_catalog.default.store_sales (38) +: +- BroadcastExchange (62) +: +- * HashAggregate (61) +: +- * CometColumnarToRow (60) +: +- CometColumnarExchange (59) +: +- * HashAggregate (58) +: +- * HashAggregate (57) +: +- * CometColumnarToRow (56) +: +- CometExchange (55) +: +- CometHashAggregate (54) +: +- CometProject (53) +: +- CometFilter (52) +: +- CometScan parquet spark_catalog.default.store_sales (51) ++- BroadcastExchange (75) + +- * HashAggregate (74) + +- * CometColumnarToRow (73) + +- CometColumnarExchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.store_sales (64) (1) CometScan parquet spark_catalog.default.store_sales @@ -95,7 +101,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) CometColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -112,314 +118,332 @@ Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_p Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [4]: [sum#6, count#7, count#8, count#12] -(9) Exchange +(9) CometColumnarExchange Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) HashAggregate [codegen id : 12] +(10) CometColumnarToRow [codegen id : 12] +Input [4]: [sum#6, count#7, count#8, count#12] + +(11) HashAggregate [codegen id : 12] Input [4]: [sum#6, count#7, count#8, 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))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#9 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#10 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] -(11) CometScan parquet spark_catalog.default.store_sales +(12) CometScan 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 +(13) 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 +(14) 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 +(15) 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) CometExchange +(16) CometExchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(16) ColumnarToRow [codegen id : 2] +(17) CometColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -(17) HashAggregate [codegen id : 2] +(18) 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 : 2] +(19) 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 +(20) CometColumnarExchange Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(20) HashAggregate [codegen id : 3] +(21) CometColumnarToRow [codegen id : 3] +Input [4]: [sum#21, count#22, count#23, count#27] + +(22) HashAggregate [codegen id : 3] 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 +(23) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 12] +(24) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(23) CometScan parquet spark_catalog.default.store_sales +(25) CometScan 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 +(26) 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 +(27) 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 +(28) 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) CometExchange +(29) CometExchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(28) ColumnarToRow [codegen id : 4] +(30) CometColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(29) HashAggregate [codegen id : 4] +(31) 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 : 4] +(32) 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 +(33) CometColumnarExchange +Input [4]: [sum#36, count#37, count#38, count#42] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(34) CometColumnarToRow [codegen id : 5] Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(32) HashAggregate [codegen id : 5] +(35) HashAggregate [codegen id : 5] 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 +(36) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 12] +(37) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(35) CometScan parquet spark_catalog.default.store_sales +(38) CometScan 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 +(39) 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 +(40) 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 +(41) 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) CometExchange +(42) CometExchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(40) ColumnarToRow [codegen id : 6] +(43) CometColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(41) HashAggregate [codegen id : 6] +(44) 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 : 6] +(45) 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 +(46) CometColumnarExchange +Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(47) CometColumnarToRow [codegen id : 7] Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 7] +(48) HashAggregate [codegen id : 7] 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 +(49) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 12] +(50) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(47) CometScan parquet spark_catalog.default.store_sales +(51) CometScan 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 +(52) 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 +(53) 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 +(54) 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) CometExchange +(55) CometExchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(52) ColumnarToRow [codegen id : 8] +(56) CometColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(53) HashAggregate [codegen id : 8] +(57) 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 : 8] +(58) 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 +(59) CometColumnarExchange Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(56) HashAggregate [codegen id : 9] +(60) CometColumnarToRow [codegen id : 9] +Input [4]: [sum#66, count#67, count#68, count#72] + +(61) HashAggregate [codegen id : 9] 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 +(62) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 12] +(63) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(59) CometScan parquet spark_catalog.default.store_sales +(64) CometScan 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 +(65) 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 +(66) 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 +(67) 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) CometExchange +(68) CometExchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(64) ColumnarToRow [codegen id : 10] +(69) CometColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(65) HashAggregate [codegen id : 10] +(70) 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 : 10] +(71) 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 +(72) CometColumnarExchange +Input [4]: [sum#81, count#82, count#83, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(73) CometColumnarToRow [codegen id : 11] Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(68) HashAggregate [codegen id : 11] +(74) HashAggregate [codegen id : 11] 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 +(75) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 12] +(76) 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 4a547c4e04..e62a830d18 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 @@ -5,95 +5,101 @@ WholeStageCodegen (12) 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 (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 - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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 (3) 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #4 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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 (5) + 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] + CometColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (2) + CometColumnarExchange #7 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #5 + CometExchange [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 #6 - WholeStageCodegen (5) - 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 + BroadcastExchange #9 + WholeStageCodegen (7) + 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] + CometColumnarToRow InputAdapter - Exchange #7 - WholeStageCodegen (4) + CometColumnarExchange #10 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #8 + CometExchange [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 #9 - WholeStageCodegen (7) - 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 + BroadcastExchange #12 + WholeStageCodegen (9) + 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] + CometColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (6) + CometColumnarExchange #13 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #11 + CometExchange [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 #12 - WholeStageCodegen (9) - 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 + BroadcastExchange #15 + WholeStageCodegen (11) + 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] + CometColumnarToRow InputAdapter - Exchange #13 - WholeStageCodegen (8) + CometColumnarExchange #16 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #14 + CometExchange [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] 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 (11) - 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 (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 - CometExchange [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] - 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-spark3_5/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt index 71d583785d..b02bf3a4cf 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -258,14 +258,14 @@ Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity# 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] +(45) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -286,7 +286,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (50) BroadcastExchange @@ -295,7 +295,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -316,7 +316,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 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] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (55) BroadcastExchange @@ -325,7 +325,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (60) -+- * ColumnarToRow (59) ++- * CometColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (60) BroadcastExchange 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 9398fcdf5e..9340a4e5f1 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 a0796a47af..e80a2eaa78 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 f82fd24bdd..7bc02651ec 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 a634822f58..83d3179285 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -268,14 +268,14 @@ Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, 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] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [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 (53) -+- * ColumnarToRow (52) ++- * CometColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (53) BroadcastExchange 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 824cc51163..2a789450e6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 76a62c8926..6ce03737eb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (90) +* CometColumnarToRow (90) +- CometSort (89) +- CometColumnarExchange (88) +- CometProject (87) @@ -504,14 +504,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT 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] -(90) ColumnarToRow [codegen id : 1] +(90) CometColumnarToRow [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 (94) -+- * ColumnarToRow (93) ++- * CometColumnarToRow (93) +- CometFilter (92) +- CometScan parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * ColumnarToRow (97) ++- * CometColumnarToRow (97) +- CometFilter (96) +- CometScan parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct 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)) -(97) ColumnarToRow [codegen id : 1] +(97) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometFilter (100) +- CometScan parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct 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)) -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange 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 d6615fe1c1..22635bb9f2 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -67,7 +67,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt index 03edc9859f..7caa3c6e24 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange 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 14b2a34a89..2650006a6c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 7a50fe69e9..587c91520a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 9556aa785b..50c2d64a92 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 50e3800526..452f25394a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 779d4cbddb..014786e7fd 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 c1e19555c9..74ff12aef1 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 9deb513423..8d4a8dcb97 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 5bd9122ffa..b10a21b708 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,7 +2,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- Window (27) - +- * ColumnarToRow (26) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometExchange (24) +- CometHashAggregate (23) @@ -148,7 +148,7 @@ Arguments: hashpartitioning(_w1#22, _w2#23, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] (27) Window @@ -167,7 +167,7 @@ Arguments: 100, [lochierarchy#20 DESC NULLS LAST, CASE WHEN (lochierarchy#20 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (34) BroadcastExchange 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 421db8dad0..5b8623c2b3 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 844e7a729f..3f78c57bb2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 85e86ca8ae..13fcba8592 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 c0e7300dff..8139d06c08 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange 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 8272260f5c..5ce896e6e6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] 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 32b231d65d..0b404d5868 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 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] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 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] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange 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 b789f3a8d2..9df61b1669 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [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 0239d76784..683ec549cc 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 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] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 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] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange 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 b789f3a8d2..9df61b1669 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [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 0fe37749f9..457cfd8a39 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (101) +* CometColumnarToRow (101) +- CometTakeOrderedAndProject (100) +- CometProject (99) +- CometBroadcastHashJoin (98) @@ -575,14 +575,14 @@ Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, custo 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] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [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 (105) -+- * ColumnarToRow (104) ++- * CometColumnarToRow (104) +- CometFilter (103) +- CometScan parquet spark_catalog.default.date_dim (102) @@ -598,7 +598,7 @@ ReadSchema: struct 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)) -(104) ColumnarToRow [codegen id : 1] +(104) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] (105) BroadcastExchange @@ -607,7 +607,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (109) -+- * ColumnarToRow (108) ++- * CometColumnarToRow (108) +- CometFilter (107) +- CometScan parquet spark_catalog.default.date_dim (106) @@ -623,7 +623,7 @@ ReadSchema: struct 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)) -(108) ColumnarToRow [codegen id : 1] +(108) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (109) BroadcastExchange 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 d6651fadcf..19f7ccde3a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -50,7 +50,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt index ef66f763d2..10640eb2bb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometTakeOrderedAndProject (31) +- CometHashAggregate (30) +- CometExchange (29) @@ -179,14 +179,14 @@ Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - 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] +(32) CometColumnarToRow [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 (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -202,7 +202,7 @@ ReadSchema: struct 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)) -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] (36) BroadcastExchange 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 eac6138196..6b87d9b8b0 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 48a1c60d16..bb3c74daeb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] 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] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [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 007103499b..de12ce9581 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] 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 b58d82c785..0ad962d86a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 40528f52d3..7307450677 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 2953a5f8f9..10dfceddc2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_s 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] +(19) CometColumnarToRow [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 447131b776..08b394e62e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 337a09591b..c8d14fd293 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,46 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Project (32) - : : +- * SortMergeJoin Inner (31) - : : :- * Sort (17) - : : : +- * Project (16) - : : : +- * Filter (15) - : : : +- Window (14) - : : : +- WindowGroupLimit (13) - : : : +- * Sort (12) - : : : +- Exchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * ColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (30) - : : +- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- WindowGroupLimit (26) - : : +- * Sort (25) - : : +- Exchange (24) - : : +- WindowGroupLimit (23) - : : +- * ColumnarToRow (22) - : : +- CometSort (21) - : : +- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- ReusedExchange (18) - : +- BroadcastExchange (36) - : +- * ColumnarToRow (35) - : +- CometFilter (34) - : +- CometScan parquet spark_catalog.default.item (33) - +- ReusedExchange (39) +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (34) + : : +- * SortMergeJoin Inner (33) + : : :- * Sort (18) + : : : +- * Project (17) + : : : +- * Filter (16) + : : : +- Window (15) + : : : +- WindowGroupLimit (14) + : : : +- * CometColumnarToRow (13) + : : : +- CometSort (12) + : : : +- CometColumnarExchange (11) + : : : +- WindowGroupLimit (10) + : : : +- * CometColumnarToRow (9) + : : : +- CometSort (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- * Sort (32) + : : +- * Project (31) + : : +- * Filter (30) + : : +- Window (29) + : : +- WindowGroupLimit (28) + : : +- * CometColumnarToRow (27) + : : +- CometSort (26) + : : +- CometColumnarExchange (25) + : : +- WindowGroupLimit (24) + : : +- * CometColumnarToRow (23) + : : +- CometSort (22) + : : +- CometFilter (21) + : : +- CometHashAggregate (20) + : : +- ReusedExchange (19) + : +- BroadcastExchange (38) + : +- * CometColumnarToRow (37) + : +- CometFilter (36) + : +- CometScan parquet spark_catalog.default.item (35) + +- ReusedExchange (41) (1) CometScan parquet spark_catalog.default.store_sales @@ -80,191 +82,197 @@ Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] (10) WindowGroupLimit Input [2]: [item_sk#7, rank_col#8] Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial -(11) Exchange +(11) CometColumnarExchange Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 2] +(12) CometSort +Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] + +(13) CometColumnarToRow [codegen id : 2] Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], false, 0 -(13) WindowGroupLimit +(14) WindowGroupLimit Input [2]: [item_sk#7, rank_col#8] Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final -(14) Window +(15) 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] -(15) Filter [codegen id : 3] +(16) Filter [codegen id : 3] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(16) Project [codegen id : 3] +(17) Project [codegen id : 3] Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] -(17) Sort [codegen id : 3] +(18) Sort [codegen id : 3] Input [2]: [item_sk#7, rnk#11] Arguments: [rnk#11 ASC NULLS FIRST], false, 0 -(18) ReusedExchange [Reuses operator id: 5] +(19) ReusedExchange [Reuses operator id: 5] Output [3]: [ss_item_sk#12, sum#13, count#14] -(19) CometHashAggregate +(20) 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))] -(20) CometFilter +(21) 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]))) -(21) CometSort +(22) CometSort Input [2]: [item_sk#16, rank_col#17] Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(22) ColumnarToRow [codegen id : 4] +(23) CometColumnarToRow [codegen id : 4] Input [2]: [item_sk#16, rank_col#17] -(23) WindowGroupLimit +(24) WindowGroupLimit Input [2]: [item_sk#16, rank_col#17] Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial -(24) Exchange +(25) CometColumnarExchange +Input [2]: [item_sk#16, rank_col#17] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(26) CometSort Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] -(25) Sort [codegen id : 5] +(27) CometColumnarToRow [codegen id : 5] Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], false, 0 -(26) WindowGroupLimit +(28) WindowGroupLimit Input [2]: [item_sk#16, rank_col#17] Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final -(27) Window +(29) 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] -(28) Filter [codegen id : 6] +(30) Filter [codegen id : 6] Input [3]: [item_sk#16, rank_col#17, rnk#18] Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) -(29) Project [codegen id : 6] +(31) Project [codegen id : 6] Output [2]: [item_sk#16, rnk#18] Input [3]: [item_sk#16, rank_col#17, rnk#18] -(30) Sort [codegen id : 6] +(32) Sort [codegen id : 6] Input [2]: [item_sk#16, rnk#18] Arguments: [rnk#18 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 9] +(33) SortMergeJoin [codegen id : 9] Left keys [1]: [rnk#11] Right keys [1]: [rnk#18] Join type: Inner Join condition: None -(32) Project [codegen id : 9] +(34) 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] -(33) CometScan parquet spark_catalog.default.item +(35) CometScan 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 -(34) CometFilter +(36) CometFilter Input [2]: [i_item_sk#19, i_product_name#20] Condition : isnotnull(i_item_sk#19) -(35) ColumnarToRow [codegen id : 7] +(37) CometColumnarToRow [codegen id : 7] Input [2]: [i_item_sk#19, i_product_name#20] -(36) BroadcastExchange +(38) 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] -(37) BroadcastHashJoin [codegen id : 9] +(39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [item_sk#7] Right keys [1]: [i_item_sk#19] Join type: Inner Join condition: None -(38) Project [codegen id : 9] +(40) 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] -(39) ReusedExchange [Reuses operator id: 36] +(41) ReusedExchange [Reuses operator id: 38] Output [2]: [i_item_sk#21, i_product_name#22] -(40) BroadcastHashJoin [codegen id : 9] +(42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [item_sk#16] Right keys [1]: [i_item_sk#21] Join type: Inner Join condition: None -(41) Project [codegen id : 9] +(43) 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] -(42) TakeOrderedAndProject +(44) 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 = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (49) -+- CometHashAggregate (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.store_sales (43) +* CometColumnarToRow (51) ++- CometHashAggregate (50) + +- CometExchange (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.store_sales (45) -(43) CometScan parquet spark_catalog.default.store_sales +(45) CometScan 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 -(44) CometFilter +(46) 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)) -(45) CometProject +(47) 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] -(46) CometHashAggregate +(48) 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))] -(47) CometExchange +(49) CometExchange Input [3]: [ss_store_sk#26, sum#29, count#30] Arguments: hashpartitioning(ss_store_sk#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometHashAggregate +(50) 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))] -(49) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#31] -Subquery:2 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] +Subquery:2 Hosting operator id = 21 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 58687dc84b..3cd0db184e 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 @@ -15,31 +15,32 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Window [rank_col] WindowGroupLimit [rank_col] WholeStageCodegen (2) - Sort [rank_col] + CometColumnarToRow InputAdapter - Exchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] - CometExchange [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))] - CometExchange [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] + CometSort [item_sk,rank_col] + CometColumnarExchange #1 + WindowGroupLimit [rank_col] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometFilter [item_sk,rank_col] + Subquery #1 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometExchange [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))] + CometExchange [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 (6) Sort [rnk] @@ -49,22 +50,23 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] Window [rank_col] WindowGroupLimit [rank_col] WholeStageCodegen (5) - Sort [rank_col] + CometColumnarToRow InputAdapter - Exchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - 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 + CometSort [item_sk,rank_col] + CometColumnarExchange #4 + WindowGroupLimit [rank_col] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + 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 (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [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 962a51203b..d785f36b6a 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,41 @@ == 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 (37) ++- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) CometScan parquet spark_catalog.default.web_sales @@ -149,7 +150,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +168,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -195,50 +196,53 @@ 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) CometColumnarExchange Input [3]: [ca_zip#11, ca_city#10, sum#20] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(35) HashAggregate [codegen id : 3] +(35) CometColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(36) 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 +(37) 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 (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(37) CometScan parquet spark_catalog.default.date_dim +(38) CometScan 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 +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(42) 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 f60fdb18a1..f43d2a0146 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,52 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 7bc10ce88e..406d3a6730 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_tick 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] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (43) BroadcastExchange 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 e1b53bf4d0..25e38a548e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] 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 32126320ff..e39a63cc69 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [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] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 cf4556774d..26c6cd226a 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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-spark3_5/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt index 70c7dc75f5..451f4d9646 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometHashAggregate (27) +- CometExchange (26) +- CometHashAggregate (25) @@ -159,14 +159,14 @@ Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -187,7 +187,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (33) BroadcastExchange 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 a46d8d6037..60c611beca 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 2def4544c3..041046c4af 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,82 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * ColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (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) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (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) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (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) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (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) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (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) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (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) CometScan parquet spark_catalog.default.web_sales @@ -172,7 +173,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] 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) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +274,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] 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) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +375,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] 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) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -406,50 +407,51 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) Exchange +(75) 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, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 11] +(76) 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#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(77) TakeOrderedAndProject +(77) 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] + +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: 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] ===== 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 (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) -(78) CometScan parquet spark_catalog.default.date_dim +(79) CometScan 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 +(80) 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] 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-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 d98b2b0a66..21827c75d8 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,109 +1,110 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (11) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - 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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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 +WholeStageCodegen (11) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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 02ad2f3575..84d57d5645 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,75 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * HashAggregate (22) - : +- * ColumnarToRow (21) - : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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 (71) ++- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (22) + : +- * CometColumnarToRow (21) + : +- CometExchange (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) + : +- * CometColumnarToRow (40) + : +- CometExchange (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) + +- * CometColumnarToRow (63) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -168,7 +169,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -259,7 +260,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled 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, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -369,7 +370,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -392,50 +393,53 @@ Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(prof 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) 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, [plan_id=4] +Arguments: hashpartitioning(channel#118, id#119, spark_grouping_id#120, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(69) HashAggregate [codegen id : 5] +(69) CometColumnarToRow [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] + +(70) 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 +(71) 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 (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(71) CometScan parquet spark_catalog.default.date_dim +(72) CometScan 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 +(73) 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 +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(76) 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 c6f9f60af4..7458de143c 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,93 +1,94 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - CometExchange [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] - 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 (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 - CometExchange [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] - 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 (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 - CometExchange [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] - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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) + CometColumnarToRow + 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 - 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] + 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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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-spark3_5/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt index 9b1efc7ada..ded5068bb0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -160,14 +160,14 @@ Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30 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] +(28) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -188,7 +188,7 @@ Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2 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] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] (33) BroadcastExchange 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 5333344640..c0a0b6d39c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 8779bd70e7..cdfe36d00c 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,45 @@ == 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) - : +- * ColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (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) - +- * ColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * Project (16) + : +- Window (15) + : +- * CometColumnarToRow (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (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 (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan parquet spark_catalog.default.web_sales @@ -103,7 +104,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (15) Window @@ -114,13 +115,13 @@ Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FI 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 +(17) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) Sort [codegen id : 3] +(18) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (19) CometScan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] @@ -168,84 +169,86 @@ Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeS 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] -(29) ColumnarToRow [codegen id : 4] +(29) CometColumnarToRow [codegen id : 3] Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] (30) 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] -(31) Project [codegen id : 5] +(31) 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] -(32) Exchange +(32) CometColumnarExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(33) Sort [codegen id : 6] +(33) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(34) SortMergeJoin [codegen id : 7] -Left keys [2]: [item_sk#9, d_date#6] -Right keys [2]: [item_sk#19, d_date#17] -Join type: FullOuter -Join condition: None +(34) 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 -(35) Project [codegen id : 7] -Output [4]: [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) 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] -(36) Exchange +(36) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) Sort [codegen id : 8] +(37) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], false, 0 +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] -(38) Window +(38) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] + +(39) Window Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [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] -(39) Filter [codegen id : 9] +(40) Filter [codegen id : 6] 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)) -(40) TakeOrderedAndProject +(41) TakeOrderedAndProject Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] Arguments: 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] ===== 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 (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(41) CometScan parquet spark_catalog.default.date_dim +(42) CometScan 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 +(43) 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 +(44) 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] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] 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-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 f972c31c17..c829621bcc 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,68 +1,62 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (9) + WholeStageCodegen (6) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (8) - Sort [item_sk,d_date] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (7) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - SortMergeJoin [item_sk,d_date,item_sk,d_date] - InputAdapter - WholeStageCodegen (3) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] - CometExchange [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] - InputAdapter - WholeStageCodegen (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] - CometExchange [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 + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometExchange [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) + CometColumnarToRow + 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 + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometExchange [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 42b974e53f..c7a400159f 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 af5223b69e..1e7168862a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 9b71fa4009..c1448cf812 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,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ 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) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 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] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange 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 363b84b354..96a2eec50e 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 @@ -5,7 +5,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] 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 73422b2923..72ba214250 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (56) +* CometColumnarToRow (56) +- CometTakeOrderedAndProject (55) +- CometHashAggregate (54) +- CometExchange (53) @@ -320,14 +320,14 @@ Functions [1]: [count(1)] 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] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometFilter (58) +- CometScan parquet spark_catalog.default.date_dim (57) @@ -348,7 +348,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 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] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] (61) BroadcastExchange @@ -359,7 +359,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -380,7 +380,7 @@ Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] (66) BroadcastExchange @@ -388,7 +388,7 @@ Input [1]: [d_date_sk#31] 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#33, [id=#34] -* ColumnarToRow (73) +* CometColumnarToRow (73) +- CometHashAggregate (72) +- CometExchange (71) +- CometHashAggregate (70) @@ -426,11 +426,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [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) +* CometColumnarToRow (80) +- CometHashAggregate (79) +- CometExchange (78) +- CometHashAggregate (77) @@ -468,7 +468,7 @@ Input [1]: [(d_month_seq + 3)#49] Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [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] 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 1290186685..e41bbe85b2 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] CometHashAggregate [segment,num_customers,segment_base,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -58,13 +58,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] CometExchange [(d_month_seq + 1)] #10 @@ -74,7 +74,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] CometExchange [(d_month_seq + 3)] #11 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 46240a3c02..4549080d86 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 999c8a6c4c..5a5f7ee21e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 c56eb8287f..e3977ede3f 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 2603a2781f..95ff3d709a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 78df07e8fb..8f45480fc8 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 39a7ce9d67..10cd515da7 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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-spark3_5/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt index cd1e112f69..18736927ab 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (51) +* CometColumnarToRow (51) +- CometTakeOrderedAndProject (50) +- CometProject (49) +- CometBroadcastHashJoin (48) @@ -280,14 +280,14 @@ Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws 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] +(51) CometColumnarToRow [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) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometBroadcastHashJoin (58) :- CometFilter (53) @@ -337,7 +337,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (61) BroadcastExchange @@ -345,7 +345,7 @@ Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* ColumnarToRow (65) +* CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) Input [2]: [d_date#40, d_week_seq#41] Arguments: [d_week_seq#41], [d_week_seq#41] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#41] Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 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 36fc8ac573..db48fd9775 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -30,7 +30,7 @@ WholeStageCodegen (1) CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_date,d_week_seq] 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 62311fb338..81f784c134 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,6 +219,6 @@ Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_ 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] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [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 140a764391..15f2e45693 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 e87f6ce763..542920b01c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) 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] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [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 05566cbca5..1ba3d55d23 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 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 0d64004d9a..5596623c61 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 4dd8b3b5bd..ea973a4810 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 4a157e64e8..9656e93142 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,7 +1,7 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * ColumnarToRow (41) + :- * CometColumnarToRow (41) : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) @@ -43,7 +43,7 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -256,7 +256,7 @@ Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -(41) ColumnarToRow [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] Input [1]: [promotions#24] (42) CometScan parquet spark_catalog.default.store_sales @@ -345,7 +345,7 @@ Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [total#38] (63) BroadcastExchange @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * ColumnarToRow (69) ++- * CometColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 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] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (70) BroadcastExchange 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 548614416e..30c16b696d 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,7 +1,7 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 @@ -23,7 +23,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -54,7 +54,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 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 c174784d38..b80eab371d 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH 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] +(28) CometColumnarToRow [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 6a9b1ced35..002eedf08e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 2a5ae3d1f4..08e1a99a58 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,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 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] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange 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 8ea16ea740..8e1ce8c997 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 @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] 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 2dcffc7a77..8fd7206781 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC 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] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] 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 (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct 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)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct 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)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange 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 49a08d6f8b..0ce55b5714 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt index e0f1b47992..09ddb55e37 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,14 +219,14 @@ Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh 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] +(39) CometColumnarToRow [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 = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -247,7 +247,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] (44) BroadcastExchange 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 be08ee23a0..28412503a2 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 693eb5662d..58f093ae40 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometTakeOrderedAndProject (49) +- CometHashAggregate (48) +- CometExchange (47) @@ -277,14 +277,14 @@ Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum 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] +(50) CometColumnarToRow [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 (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -300,7 +300,7 @@ ReadSchema: struct 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)) -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] (54) BroadcastExchange 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 a2b5963e13..3529a210a6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt index d63914c5e6..1cf7a9c7a8 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,36 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Filter (30) - +- Window (29) - +- WindowGroupLimit (28) - +- * Sort (27) - +- Exchange (26) - +- WindowGroupLimit (25) - +- * ColumnarToRow (24) - +- CometSort (23) - +- CometHashAggregate (22) - +- CometExchange (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 (32) ++- * Filter (31) + +- Window (30) + +- WindowGroupLimit (29) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- WindowGroupLimit (25) + +- * CometColumnarToRow (24) + +- CometSort (23) + +- CometHashAggregate (22) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -142,66 +143,69 @@ Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(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] 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] +(24) CometColumnarToRow [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#30] Arguments: [i_category#19], [sumsales#30 DESC NULLS LAST], rank(sumsales#30), 100, Partial -(26) Exchange +(26) 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, [plan_id=2] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(27) Sort [codegen id : 2] +(27) 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] + +(28) CometColumnarToRow [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] -Arguments: [i_category#19 ASC NULLS FIRST, sumsales#30 DESC NULLS LAST], false, 0 -(28) WindowGroupLimit +(29) 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 +(30) 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 : 3] +(31) 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 +(32) 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 (37) ++- * CometColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(32) CometScan parquet spark_catalog.default.date_dim +(33) CometScan 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 +(34) 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 +(35) 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] +(36) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(36) BroadcastExchange +(37) 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 1f3b8c203c..6877c2f761 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 @@ -5,41 +5,42 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] WholeStageCodegen (2) - Sort [i_category,sumsales] + CometColumnarToRow InputAdapter - Exchange [i_category] #1 - 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))] - CometExchange [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] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + CometColumnarToRow + 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))] + CometExchange [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) + CometColumnarToRow + 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-spark3_5/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt index 121987260e..00c0db86ec 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_tick 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] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (43) BroadcastExchange 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 568fd734b9..986918c8b4 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 5386a3d166..d1080a5599 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +187,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(50) 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 c9cc4959b7..09ef18a99b 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) - 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_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] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + CometColumnarToRow + 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) + CometColumnarToRow + 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] 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 d18a60b0b4..adc27c2e86 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns 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] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange 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 044813ad42..96e7e32f95 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 8e3afee025..bb63bc6750 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,51 @@ == 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) - +- * ColumnarToRow (27) - +- CometSort (26) - +- CometHashAggregate (25) - +- CometExchange (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 (47) ++- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- CometColumnarExchange (42) + +- * HashAggregate (41) + +- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * Expand (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * CometColumnarToRow (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) + :- * CometColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- Window (29) + +- WindowGroupLimit (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometHashAggregate (25) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -86,7 +88,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 : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -100,7 +102,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 : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -169,7 +171,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] 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] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit @@ -223,66 +225,72 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#23] Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(39) Exchange +(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, [plan_id=4] +Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 5] +(40) CometColumnarToRow [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))#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] -(41) Exchange +(42) 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] + +(43) CometSort 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, [plan_id=5] +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 : 6] +(44) CometColumnarToRow [codegen id : 6] Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] -Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0 -(43) Window +(45) 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 : 7] +(46) 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 +(47) 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 (52) ++- * CometColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) CometScan parquet spark_catalog.default.date_dim +(48) CometScan 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 +(49) 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 +(50) 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] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(50) BroadcastExchange +(52) 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 7da5aad4f8..34addbb62d 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 @@ -4,67 +4,69 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (6) - Sort [_w1,_w2,_w0] + CometColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (5) - 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 (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 - 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 (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 #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))] - CometExchange [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 + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + 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) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometExchange [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 79a71af03a..fe8f7c5f76 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (37) +* CometColumnarToRow (37) +- CometSort (36) +- CometColumnarExchange (35) +- CometHashAggregate (34) @@ -209,14 +209,14 @@ Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS 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) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (42) -+- * ColumnarToRow (41) ++- * CometColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11 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] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (42) BroadcastExchange 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 88ca8b67df..7214ec67a5 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [ext_price,brand_id] #1 @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 6aae88fff3..4179f3b1d6 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 b0ca5520f2..a7c93664a7 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 4417365325..eaede7232d 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 c7a559c8ef..6358cf7e31 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange 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 631a82f1e3..9bde9ac863 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt index 2f33f50a8b..9d68c7a66c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[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]), [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], 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] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct 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)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct 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)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange 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 9e340130a5..5a8af58abc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt index a2a17ccfce..24ed6809af 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -190,6 +190,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] 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] +(34) CometColumnarToRow [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 eb3e14d5ec..3e89713523 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 661b23c47b..55f5778124 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,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * HashAggregate (83) - +- Exchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * ColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (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) - : +- CometExchange (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) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * ColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * Expand (80) + +- Union (79) + :- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometExchange (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) + : +- CometExchange (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) + : : +- * CometColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : +- ReusedExchange (33) + : +- * CometColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : +- ReusedExchange (42) + +- * CometColumnarToRow (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometHashAggregate (63) + : +- CometExchange (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) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -226,7 +227,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight 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] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -262,7 +263,7 @@ 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))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -302,7 +303,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -438,7 +439,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight 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] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] (79) Union @@ -454,50 +455,53 @@ Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(prof 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] -(82) Exchange +(82) 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, [plan_id=8] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(83) HashAggregate [codegen id : 6] +(83) CometColumnarToRow [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] + +(84) 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] -(84) TakeOrderedAndProject +(85) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan parquet spark_catalog.default.date_dim (85) +BroadcastExchange (90) ++- * CometColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan parquet spark_catalog.default.date_dim (86) -(85) CometScan parquet spark_catalog.default.date_dim +(86) CometScan 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 -(86) CometFilter +(87) 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)) -(87) CometProject +(88) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(88) ColumnarToRow [codegen id : 1] +(89) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(89) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#6] 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-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 e8226ab1fa..d9fa7d8f2e 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,110 +1,111 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - 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))] - CometExchange [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] - 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] - 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))] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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) + CometColumnarToRow 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))] - CometExchange [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 - ColumnarToRow + 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))] + CometExchange [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] + 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) + CometColumnarToRow + 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] + 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))] + CometExchange [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 - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #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 (4) - 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))] - CometExchange [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] - 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] - 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))] - CometExchange [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] + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [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 + CometColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #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 (4) + CometColumnarToRow + 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))] + CometExchange [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] + 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 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk] #12 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + 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))] + CometExchange [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 b8b24b9d0b..41ea1060b6 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,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss 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] +(65) CometColumnarToRow [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] @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct 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)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange 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 3318702eff..6b85e03ae9 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,7 +1,7 @@ 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 (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - ColumnarToRow + CometColumnarToRow InputAdapter 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] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt index e697c9038b..c4b54c4ed5 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometProject (28) +- CometBroadcastHashJoin (27) @@ -169,14 +169,14 @@ Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket 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] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (35) BroadcastExchange 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 b69b0cd2e9..d5514ba8a7 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] 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 f02edb1912..c6f5a077fa 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -236,14 +236,14 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] 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] +(42) CometColumnarToRow [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 (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -264,7 +264,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (47) BroadcastExchange 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 91d98aca9b..7beec478b3 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] 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 65fe1a28b0..d902d76cd2 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,106 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (102) -+- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Expand (98) - +- Union (97) - :- * HashAggregate (38) - : +- * ColumnarToRow (37) - : +- CometExchange (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) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (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) - : +- CometExchange (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) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (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) - +- CometExchange (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) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (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) +TakeOrderedAndProject (103) ++- * HashAggregate (102) + +- * CometColumnarToRow (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- * Expand (98) + +- Union (97) + :- * HashAggregate (38) + : +- * CometColumnarToRow (37) + : +- CometExchange (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) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (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) + : +- * CometColumnarToRow (66) + : +- CometExchange (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) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (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) + +- * CometColumnarToRow (95) + +- CometExchange (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) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -272,7 +273,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -403,7 +404,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c 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, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [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] @@ -534,7 +535,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c 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, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -557,50 +558,53 @@ Functions [3]: [partial_sum(sales#30), partial_sum(returns#31), partial_sum(prof 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) Exchange +(100) 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, [plan_id=10] +Arguments: hashpartitioning(channel#97, id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(101) HashAggregate [codegen id : 5] +(101) CometColumnarToRow [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] + +(102) 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] -(102) TakeOrderedAndProject +(103) 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 (107) -+- * ColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan parquet spark_catalog.default.date_dim (103) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(103) CometScan parquet spark_catalog.default.date_dim +(104) CometScan 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 -(104) CometFilter +(105) 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)) -(105) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(107) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] 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-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 012c711bfe..981b9671d1 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,122 +1,123 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 - CometExchange [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] - CometExchange [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] - CometExchange [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 - CometExchange [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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 10e1267143..cf88aa2e38 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometTakeOrderedAndProject (46) +- CometProject (45) +- CometBroadcastHashJoin (44) @@ -263,14 +263,14 @@ Arguments: [c_customer_id#29, c_salutation#31, c_first_name#32, c_last_name#33, 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (52) BroadcastExchange 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 b35b762107..c670e94552 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 34319a8fc3..6117d4a921 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 42c3f40e68..755128d2c4 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 c0dfa3fdf0..820bc9bd5c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -265,14 +265,14 @@ Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr 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] +(48) CometColumnarToRow [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) ++- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (50) @@ -344,7 +344,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (62) BroadcastExchange 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 77345d28a7..0530aad2f4 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 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 c03da66a07..020414c4d7 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -256,14 +256,14 @@ Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg 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] +(45) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -284,7 +284,7 @@ Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] (50) BroadcastExchange 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 4e4ece1bda..28752f3783 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -25,7 +25,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 cf40b3e8d5..e7ce31dd2b 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,7 +2,7 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,7 +114,7 @@ Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (21) Window @@ -133,7 +133,7 @@ Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (28) BroadcastExchange 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 86480d58c6..b1bf4c2e38 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -21,7 +21,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 28b942e2f7..d496a7ecb2 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,54 @@ == Physical Plan == -* HashAggregate (49) -+- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin LeftAnti BuildRight (45) - :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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 (50) ++- * CometColumnarToRow (49) + +- CometColumnarExchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * CometColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometExchange (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) + : +- * CometColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometExchange (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) + +- * CometColumnarToRow (43) + +- CometHashAggregate (42) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -128,7 +129,7 @@ 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: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +182,7 @@ 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: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +245,7 @@ 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: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -268,11 +269,14 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) CometColumnarExchange Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) HashAggregate [codegen id : 4] +(49) CometColumnarToRow [codegen id : 4] +Input [1]: [count#27] + +(50) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -282,32 +286,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 (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(50) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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 +(53) 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] +(54) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(55) 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 6e3328ae1d..e4f27f3a1b 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,69 +1,70 @@ WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (3) + HashAggregate [count,count] + Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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) + CometColumnarToRow + 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 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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] + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 + CometExchange [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 [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] + 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 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt index 592e23cd27..6682e151f1 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,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange 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 1bb61b6c92..21b72d36a7 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 @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 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 143ff08683..8346f9b848 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,7 +3,7 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometSort (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -139,7 +139,7 @@ Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_ 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) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [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 @@ -162,7 +162,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_ 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] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] (33) BroadcastExchange 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 97b9563be6..58bb7e979e 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 @@ -5,7 +5,7 @@ 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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] 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 8261f96537..831b71a98c 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 @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (4) +* CometColumnarToRow (4) +- CometProject (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -20,13 +20,13 @@ Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) Input [1]: [r_reason_sk#1] Arguments: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6], [CASE WHEN (Subquery scalar-subquery#7, [id=#8].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_net_paid) END AS bucket1#2, CASE WHEN (Subquery scalar-subquery#9, [id=#10].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_net_paid) END AS bucket2#3, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket3#4, CASE WHEN (Subquery scalar-subquery#13, [id=#14].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_net_paid) END AS bucket4#5, CASE WHEN (Subquery scalar-subquery#15, [id=#16].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_net_paid) END AS bucket5#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [5]: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -* ColumnarToRow (12) +* CometColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal 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] +(12) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] Subquery:4 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (20) +* CometColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] Subquery:7 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal 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] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:10 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* ColumnarToRow (36) +* CometColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] Subquery:13 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#15, [id=#16] -* ColumnarToRow (44) +* CometColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#15, [id=#16] 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 cfeb9ebe74..3e1b996a51 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,10 +1,10 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [bucket1,bucket2,bucket3,bucket4,bucket5] Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -17,7 +17,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -30,7 +30,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -43,7 +43,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -56,7 +56,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 a6ec6f4b96..38379fb61c 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,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange 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 95fd73d86e..72e45a9ad1 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,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 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 ede0953066..26272decdb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometSort (40) +- CometColumnarExchange (39) +- CometHashAggregate (38) @@ -232,14 +232,14 @@ Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIRE 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) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1 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] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (46) BroadcastExchange 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 0dc197b1b4..1847051c7f 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [Returns_Loss] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 93bb03fae1..25045775b0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange 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 f49dd1ea30..4ed5b0b8ef 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 dc64f3c4ce..ae826e9288 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (22) +* CometColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] 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] +(22) CometColumnarToRow [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 6795d7e399..9580e56671 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] 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 4dd9246cd7..54db114394 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * ColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.web_site (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * CometColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.web_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.web_site (29) (1) CometScan parquet spark_catalog.default.web_sales @@ -203,7 +204,7 @@ 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))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -220,11 +221,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] Results [3]: [sum#20, sum#21, count#25] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#20, sum#21, count#25] + +(41) HashAggregate [codegen id : 2] Input [3]: [sum#20, sum#21, 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 601f577da9..9001c8e061 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - 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] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [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_order_number,ws_warehouse_sk] + 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] + CometExchange [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] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [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] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 986abf83ca..7fc71a21b0 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,57 +1,58 @@ == Physical Plan == -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.web_site (42) +* HashAggregate (54) ++- * CometColumnarToRow (53) + +- CometColumnarExchange (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * CometColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometBroadcastHashJoin (46) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometSortMergeJoin (29) + : : : :- CometSortMergeJoin (15) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometProject (14) + : : : : +- CometSortMergeJoin (13) + : : : : :- CometSort (10) + : : : : : +- CometExchange (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : : +- CometSort (12) + : : : : +- ReusedExchange (11) + : : : +- CometProject (28) + : : : +- CometSortMergeJoin (27) + : : : :- CometSort (20) + : : : : +- CometExchange (19) + : : : : +- CometProject (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.web_returns (16) + : : : +- CometProject (26) + : : : +- CometSortMergeJoin (25) + : : : :- CometSort (22) + : : : : +- ReusedExchange (21) + : : : +- CometSort (24) + : : : +- ReusedExchange (23) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.date_dim (30) + : +- CometBroadcastExchange (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer_address (36) + +- CometBroadcastExchange (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.web_site (42) (1) CometScan parquet spark_catalog.default.web_sales @@ -270,7 +271,7 @@ 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))] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, sum#25, sum#26] (50) HashAggregate [codegen id : 1] @@ -287,11 +288,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] Results [3]: [sum#25, sum#26, count#30] -(52) Exchange +(52) CometColumnarExchange Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) HashAggregate [codegen id : 2] +(53) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#25, sum#26, count#30] + +(54) HashAggregate [codegen id : 2] Input [3]: [sum#25, sum#26, 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 168f353a7b..b1aaa8fc44 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,57 +1,58 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [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] CometProject [ws_order_number] CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometExchange [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] CometSort [ws_warehouse_sk,ws_order_number] ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - 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 [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [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] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 d2e63bee25..677abdadae 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 d9a87aa3c4..9e86a025ee 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 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 1c6e9b78c0..f23cf90e66 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -139,14 +139,14 @@ 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (30) BroadcastExchange 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 0036a4bd84..aa5be145e9 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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)] CometExchange #1 @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 593065ba0b..8c2521b90c 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,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometProject (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -112,7 +112,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -135,14 +135,14 @@ Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve 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] -(25) ColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [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 (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (30) BroadcastExchange 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 01aa117344..a77e84b60b 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,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow 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] @@ -9,7 +9,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 @@ -25,7 +25,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 6dfcf8b322..b4c43d5d01 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH 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] +(28) CometColumnarToRow [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 51599575db..e112972b12 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 d47c4341a4..1cb28eaca3 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,50 @@ == 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 (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Filter (14) + : : : +- * HashAggregate (13) + : : : +- * CometColumnarToRow (12) + : : : +- CometColumnarExchange (11) + : : : +- * HashAggregate (10) + : : : +- * CometColumnarToRow (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 (30) + : : +- * Filter (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * HashAggregate (24) + : : +- * CometColumnarToRow (23) + : : +- CometColumnarExchange (22) + : : +- * HashAggregate (21) + : : +- * CometColumnarToRow (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store_returns (15) + : : +- ReusedExchange (17) + : +- BroadcastExchange (37) + : +- * CometColumnarToRow (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.store (33) + +- BroadcastExchange (43) + +- * CometColumnarToRow (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.customer (40) (1) CometScan parquet spark_catalog.default.store_returns @@ -84,7 +87,7 @@ Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] (10) HashAggregate [codegen id : 1] @@ -94,22 +97,25 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) HashAggregate [codegen id : 7] +(12) CometColumnarToRow [codegen id : 7] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#9] + +(13) 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] +(14) 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) CometScan parquet spark_catalog.default.store_returns +(15) CometScan 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 +123,181 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#17), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) CometFilter +(16) 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] +(17) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#19] -(17) CometBroadcastHashJoin +(18) 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 +(19) 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] +(20) CometColumnarToRow [codegen id : 2] Input [3]: [sr_customer_sk#14, sr_store_sk#15, sr_return_amt#16] -(20) HashAggregate [codegen id : 2] +(21) 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 +(22) 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] + +(23) CometColumnarToRow [codegen id : 3] 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) 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] +(25) 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 +(26) CometColumnarExchange +Input [3]: [ctr_store_sk#22, sum#26, count#27] +Arguments: hashpartitioning(ctr_store_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(27) CometColumnarToRow [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] +(28) 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] +(29) 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 +(30) 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] +(31) 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] +(32) 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) CometScan parquet spark_catalog.default.store +(33) CometScan 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 +(34) 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 +(35) CometProject Input [2]: [s_store_sk#30, s_state#31] Arguments: [s_store_sk#30], [s_store_sk#30] -(33) ColumnarToRow [codegen id : 5] +(36) CometColumnarToRow [codegen id : 5] Input [1]: [s_store_sk#30] -(34) BroadcastExchange +(37) 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] +(38) 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] +(39) 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) CometScan parquet spark_catalog.default.customer +(40) CometScan 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 +(41) CometFilter Input [2]: [c_customer_sk#32, c_customer_id#33] Condition : isnotnull(c_customer_sk#32) -(39) ColumnarToRow [codegen id : 6] +(42) CometColumnarToRow [codegen id : 6] Input [2]: [c_customer_sk#32, c_customer_id#33] -(40) BroadcastExchange +(43) 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] +(44) 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] +(45) 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 +(46) 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 (51) ++- * CometColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(44) CometScan parquet spark_catalog.default.date_dim +(47) CometScan 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 +(48) 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 +(49) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(47) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(48) BroadcastExchange +(51) 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 = 15 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 43bad5d70e..7caa9af9e5 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,54 +8,57 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 + WholeStageCodegen (1) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [sr_customer_sk,sr_store_sk] #6 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + CometColumnarToRow + 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) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [s_store_sk] CometFilter [s_store_sk,s_state] @@ -63,7 +66,7 @@ TakeOrderedAndProject [c_customer_id] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter 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-spark4_0/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt index e32e86f270..3c2df96933 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,50 @@ == 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 (46) ++- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * CometColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) CometScan parquet spark_catalog.default.customer @@ -101,7 +102,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +124,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +156,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +192,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +220,7 @@ ReadSchema: struct -(47) CometFilter +(48) 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 +(49) 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] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) 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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt index efd4b187dd..bc0ed49101 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,72 @@ 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + 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) + CometColumnarToRow 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 #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 f141580925..3a2978b5e9 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,80 @@ == 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 (76) ++- * Project (75) + +- * BroadcastHashJoin Inner BuildRight (74) + :- * Project (57) + : +- * BroadcastHashJoin Inner BuildRight (56) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * CometColumnarToRow (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 (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.date_dim (26) + : +- BroadcastExchange (55) + : +- * Filter (54) + : +- * HashAggregate (53) + : +- * CometColumnarToRow (52) + : +- CometColumnarExchange (51) + : +- * HashAggregate (50) + : +- * CometColumnarToRow (49) + : +- CometProject (48) + : +- CometBroadcastHashJoin (47) + : :- CometProject (45) + : : +- CometBroadcastHashJoin (44) + : : :- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.customer (39) + : : +- CometBroadcastExchange (43) + : : +- CometFilter (42) + : : +- CometScan parquet spark_catalog.default.web_sales (41) + : +- ReusedExchange (46) + +- BroadcastExchange (73) + +- * HashAggregate (72) + +- * CometColumnarToRow (71) + +- CometColumnarExchange (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometProject (67) + +- CometBroadcastHashJoin (66) + :- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.customer (58) + : +- CometBroadcastExchange (62) + : +- CometFilter (61) + : +- CometScan parquet spark_catalog.default.web_sales (60) + +- ReusedExchange (65) (1) CometScan parquet spark_catalog.default.customer @@ -133,7 +137,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] (14) HashAggregate [codegen id : 1] @@ -143,33 +147,36 @@ 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) 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, [plan_id=1] +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] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarToRow [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] + +(17) 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] +(18) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(18) CometScan parquet spark_catalog.default.customer +(19) CometScan 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 +(20) 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) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +184,97 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(22) 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 +(23) 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 +(24) 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 +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometBroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: [d_date_sk#34, d_year#35] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) CometColumnarToRow [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] +(32) 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 +(33) 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] + +(34) CometColumnarToRow [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] +(35) 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 +(36) 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] +(37) 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] +(38) 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) CometScan parquet spark_catalog.default.customer +(39) CometScan 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 +(40) 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) CometScan parquet spark_catalog.default.web_sales +(41) CometScan 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 +282,89 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(40) CometFilter +(42) 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 +(43) 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 +(44) 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 +(45) 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] +(46) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#54, d_year#55] -(45) CometBroadcastHashJoin +(47) 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 +(48) 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] +(49) CometColumnarToRow [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] +(50) 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 +(51) 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, [plan_id=4] +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] -(50) HashAggregate [codegen id : 5] +(52) CometColumnarToRow [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] + +(53) 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] +(54) 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 +(55) 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] +(56) 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] +(57) 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) CometScan parquet spark_catalog.default.customer +(58) CometScan 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 +(59) 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) CometScan parquet spark_catalog.default.web_sales +(60) CometScan 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 +372,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(58) CometFilter +(61) 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 +(62) 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 +(63) 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 +(64) 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] +(65) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#74, d_year#75] -(63) CometBroadcastHashJoin +(66) 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 +(67) 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] +(68) CometColumnarToRow [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] +(69) 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 +(70) 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] + +(71) CometColumnarToRow [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] +(72) 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 +(73) 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] +(74) 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] +(75) 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 +(76) 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 (80) ++- * CometColumnarToRow (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) -(73) CometScan parquet spark_catalog.default.date_dim +(77) CometScan 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 +(78) 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] +(79) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(76) BroadcastExchange +(80) 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 = 21 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (84) ++- * CometColumnarToRow (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) -(77) CometScan parquet spark_catalog.default.date_dim +(81) CometScan 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 +(82) 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] +(83) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(80) BroadcastExchange +(84) 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 = 41 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 = 60 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 240d88bb7c..9140144ead 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,100 +8,104 @@ 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 7548befa10..75a4f4a76c 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,28 @@ == 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 (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.web_sales @@ -87,7 +89,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -97,66 +99,72 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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 +(24) 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) CometScan parquet spark_catalog.default.date_dim +(25) CometScan 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 +(26) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject +(27) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=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 1640fa3b02..b3111e0e5e 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,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 b4a7b87249..4955a3db94 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,38 @@ == 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 (34) ++- * CometColumnarToRow (33) + +- CometColumnarExchange (32) + +- * HashAggregate (31) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +175,7 @@ Arguments: [ss_hdemo_sk#2], [hd_demo_sk#21], Inner, (((((((cd_marital_status#19 Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#19, cd_education_status#20, hd_demo_sk#21, hd_dep_count#22] Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] (31) HashAggregate [codegen id : 1] @@ -184,11 +185,14 @@ 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) CometColumnarExchange Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarToRow [codegen id : 2] +Input [7]: [sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36] + +(34) 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 +202,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 (39) ++- * CometColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) -(34) CometScan parquet spark_catalog.default.date_dim +(35) CometScan 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 +(36) 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 +(37) CometProject Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(38) BroadcastExchange +(39) 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 cc52bb3230..59abf992c3 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,46 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 9649f8b07a..e06aef766f 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,108 +1,112 @@ == Physical Plan == -TakeOrderedAndProject (104) -+- * HashAggregate (103) - +- Exchange (102) - +- * HashAggregate (101) - +- * Expand (100) - +- Union (99) - :- * Project (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- Exchange (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) - : : : : +- CometExchange (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 (82) - : +- * Filter (81) - : +- * HashAggregate (80) - : +- Exchange (79) - : +- * HashAggregate (78) - : +- * ColumnarToRow (77) - : +- CometProject (76) - : +- CometBroadcastHashJoin (75) - : :- CometProject (73) - : : +- CometBroadcastHashJoin (72) - : : :- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (71) - : +- ReusedExchange (74) - +- * Project (98) - +- * Filter (97) - +- * HashAggregate (96) - +- Exchange (95) - +- * HashAggregate (94) - +- * ColumnarToRow (93) - +- CometProject (92) - +- CometBroadcastHashJoin (91) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometBroadcastHashJoin (86) - : : :- CometFilter (84) - : : : +- CometScan parquet spark_catalog.default.web_sales (83) - : : +- ReusedExchange (85) - : +- ReusedExchange (87) - +- ReusedExchange (90) +TakeOrderedAndProject (108) ++- * HashAggregate (107) + +- * CometColumnarToRow (106) + +- CometColumnarExchange (105) + +- * HashAggregate (104) + +- * Expand (103) + +- Union (102) + :- * Project (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (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) + : : : : +- CometExchange (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 (84) + : +- * Filter (83) + : +- * HashAggregate (82) + : +- * CometColumnarToRow (81) + : +- CometColumnarExchange (80) + : +- * HashAggregate (79) + : +- * CometColumnarToRow (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) + +- * Project (101) + +- * Filter (100) + +- * HashAggregate (99) + +- * CometColumnarToRow (98) + +- CometColumnarExchange (97) + +- * HashAggregate (96) + +- * CometColumnarToRow (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) (1) CometScan parquet spark_catalog.default.store_sales @@ -390,7 +394,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight 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] +(61) CometColumnarToRow [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] @@ -400,26 +404,29 @@ Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_pric 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) Exchange +(63) 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, [plan_id=2] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(64) HashAggregate [codegen id : 2] +(64) CometColumnarToRow [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(65) 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))#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] -(65) Filter [codegen id : 2] +(66) 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)))) -(66) Project [codegen id : 2] +(67) 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] -(67) CometScan parquet spark_catalog.default.catalog_sales +(68) CometScan 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 [] @@ -427,72 +434,75 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#65), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(68) CometFilter +(69) 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) -(69) ReusedExchange [Reuses operator id: 46] +(70) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#67] -(70) CometBroadcastHashJoin +(71) 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 -(71) ReusedExchange [Reuses operator id: 52] +(72) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] -(72) CometBroadcastHashJoin +(73) 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 -(73) CometProject +(74) 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] -(74) ReusedExchange [Reuses operator id: 58] +(75) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#72] -(75) CometBroadcastHashJoin +(76) 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 -(76) CometProject +(77) 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] -(77) ColumnarToRow [codegen id : 3] +(78) CometColumnarToRow [codegen id : 3] Input [5]: [cs_quantity#63, cs_list_price#64, i_brand_id#69, i_class_id#70, i_category_id#71] -(78) HashAggregate [codegen id : 3] +(79) 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] -(79) Exchange +(80) 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, [plan_id=3] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(80) HashAggregate [codegen id : 4] +(81) CometColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#76, isEmpty#77, count#78] + +(82) 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] -(81) Filter [codegen id : 4] +(83) 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)))) -(82) Project [codegen id : 4] +(84) 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] -(83) CometScan parquet spark_catalog.default.web_sales +(85) CometScan 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 [] @@ -500,267 +510,277 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(84) CometFilter +(86) 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) -(85) ReusedExchange [Reuses operator id: 46] +(87) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#89] -(86) CometBroadcastHashJoin +(88) 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 -(87) ReusedExchange [Reuses operator id: 52] +(89) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93] -(88) CometBroadcastHashJoin +(90) 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 -(89) CometProject +(91) 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] -(90) ReusedExchange [Reuses operator id: 58] +(92) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#94] -(91) CometBroadcastHashJoin +(93) 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 -(92) CometProject +(94) 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] -(93) ColumnarToRow [codegen id : 5] +(95) CometColumnarToRow [codegen id : 5] Input [5]: [ws_quantity#85, ws_list_price#86, i_brand_id#91, i_class_id#92, i_category_id#93] -(94) HashAggregate [codegen id : 5] +(96) 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] -(95) Exchange +(97) 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] + +(98) CometColumnarToRow [codegen id : 6] 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, [plan_id=4] -(96) HashAggregate [codegen id : 6] +(99) 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] -(97) Filter [codegen id : 6] +(100) 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)))) -(98) Project [codegen id : 6] +(101) 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] -(99) Union +(102) Union -(100) Expand [codegen id : 7] +(103) 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] -(101) HashAggregate [codegen id : 7] +(104) 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] -(102) Exchange +(105) 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] + +(106) CometColumnarToRow [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] -Arguments: hashpartitioning(channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(103) HashAggregate [codegen id : 8] +(107) 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] -(104) TakeOrderedAndProject +(108) 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 = 65 Hosting Expression = Subquery scalar-subquery#56, [id=#57] -* HashAggregate (121) -+- Exchange (120) - +- * HashAggregate (119) - +- * ColumnarToRow (118) - +- CometUnion (117) - :- CometProject (108) - : +- CometBroadcastHashJoin (107) - : :- CometScan parquet spark_catalog.default.store_sales (105) - : +- ReusedExchange (106) - :- CometProject (112) - : +- CometBroadcastHashJoin (111) - : :- CometScan parquet spark_catalog.default.catalog_sales (109) - : +- ReusedExchange (110) - +- CometProject (116) - +- CometBroadcastHashJoin (115) - :- CometScan parquet spark_catalog.default.web_sales (113) - +- ReusedExchange (114) - - -(105) CometScan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +* HashAggregate (126) ++- * CometColumnarToRow (125) + +- CometColumnarExchange (124) + +- * HashAggregate (123) + +- * CometColumnarToRow (122) + +- CometUnion (121) + :- CometProject (112) + : +- CometBroadcastHashJoin (111) + : :- CometScan parquet spark_catalog.default.store_sales (109) + : +- ReusedExchange (110) + :- CometProject (116) + : +- CometBroadcastHashJoin (115) + : :- CometScan parquet spark_catalog.default.catalog_sales (113) + : +- ReusedExchange (114) + +- CometProject (120) + +- CometBroadcastHashJoin (119) + :- CometScan parquet spark_catalog.default.web_sales (117) + +- ReusedExchange (118) + + +(109) CometScan 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#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#124)] ReadSchema: struct -(106) ReusedExchange [Reuses operator id: 19] +(110) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#125] -(107) CometBroadcastHashJoin +(111) 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 -(108) CometProject +(112) 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] -(109) CometScan parquet spark_catalog.default.catalog_sales +(113) CometScan 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#130), dynamicpruningexpression(cs_sold_date_sk#130 IN dynamicpruning#131)] ReadSchema: struct -(110) ReusedExchange [Reuses operator id: 19] +(114) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#132] -(111) CometBroadcastHashJoin +(115) 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 -(112) CometProject +(116) 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] -(113) CometScan parquet spark_catalog.default.web_sales +(117) CometScan 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#137), dynamicpruningexpression(ws_sold_date_sk#137 IN dynamicpruning#138)] ReadSchema: struct -(114) ReusedExchange [Reuses operator id: 19] +(118) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#139] -(115) CometBroadcastHashJoin +(119) 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 -(116) CometProject +(120) 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] -(117) CometUnion +(121) 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] -(118) ColumnarToRow [codegen id : 1] +(122) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#126, list_price#127] -(119) HashAggregate [codegen id : 1] +(123) HashAggregate [codegen id : 1] Input [2]: [quantity#126, list_price#127] Keys: [] 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] -(120) Exchange +(124) CometColumnarExchange Input [2]: [sum#144, count#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(121) HashAggregate [codegen id : 2] +(125) CometColumnarToRow [codegen id : 2] +Input [2]: [sum#144, count#145] + +(126) HashAggregate [codegen id : 2] Input [2]: [sum#144, count#145] Keys: [] 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 = 105 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 109 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 109 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 113 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 113 Hosting Expression = ws_sold_date_sk#137 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 117 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 (126) -+- * ColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometScan parquet spark_catalog.default.date_dim (122) +BroadcastExchange (131) ++- * CometColumnarToRow (130) + +- CometProject (129) + +- CometFilter (128) + +- CometScan parquet spark_catalog.default.date_dim (127) -(122) CometScan parquet spark_catalog.default.date_dim +(127) CometScan 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 -(123) CometFilter +(128) 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)) -(124) CometProject +(129) CometProject Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(125) ColumnarToRow [codegen id : 1] +(130) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(126) BroadcastExchange +(131) BroadcastExchange Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (131) -+- * ColumnarToRow (130) - +- CometProject (129) - +- CometFilter (128) - +- CometScan parquet spark_catalog.default.date_dim (127) +BroadcastExchange (136) ++- * CometColumnarToRow (135) + +- CometProject (134) + +- CometFilter (133) + +- CometScan parquet spark_catalog.default.date_dim (132) -(127) CometScan parquet spark_catalog.default.date_dim +(132) CometScan 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 -(128) CometFilter +(133) 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)) -(129) CometProject +(134) CometProject Input [2]: [d_date_sk#26, d_year#148] Arguments: [d_date_sk#26], [d_date_sk#26] -(130) ColumnarToRow [codegen id : 1] +(135) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(131) BroadcastExchange +(136) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] @@ -768,12 +788,12 @@ Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dy 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#56, [id=#57] +Subquery:9 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] -Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#65 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#65 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 97 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] +Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#56, [id=#57] -Subquery:12 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 85 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 e0e466c187..7099d8b2a6 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,167 +1,172 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] 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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #15 + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #15 + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 WholeStageCodegen (1) - HashAggregate [quantity,list_price] [sum,count,sum,count] - ColumnarToRow + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow 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 (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 #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] - CometExchange [brand_id,class_id,category_id] #6 + 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) + CometColumnarToRow + 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] - 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] + CometExchange [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) + CometColumnarToRow + 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_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] - 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] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #16 - 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] #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] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #17 - 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] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 + 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (3) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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] #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 + WholeStageCodegen (5) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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] #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 e4a215cd28..be253bfe04 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,90 +1,92 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * BroadcastHashJoin Inner BuildRight (85) - :- * Filter (65) - : +- * HashAggregate (64) - : +- Exchange (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) - : : : : +- CometExchange (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 (84) - +- * Filter (83) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * ColumnarToRow (79) - +- CometProject (78) - +- CometBroadcastHashJoin (77) - :- CometProject (72) - : +- CometBroadcastHashJoin (71) - : :- CometBroadcastHashJoin (69) - : : :- CometFilter (67) - : : : +- CometScan parquet spark_catalog.default.store_sales (66) - : : +- ReusedExchange (68) - : +- ReusedExchange (70) - +- CometBroadcastExchange (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +TakeOrderedAndProject (88) ++- * BroadcastHashJoin Inner BuildRight (87) + :- * Filter (66) + : +- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (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) + : : : : +- CometExchange (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 (86) + +- * Filter (85) + +- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * CometColumnarToRow (80) + +- CometProject (79) + +- CometBroadcastHashJoin (78) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (70) + : : :- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (69) + : +- ReusedExchange (71) + +- CometBroadcastExchange (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) (1) CometScan parquet spark_catalog.default.store_sales @@ -372,7 +374,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight 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] +(61) CometColumnarToRow [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] @@ -382,22 +384,25 @@ Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_pric 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] -(63) Exchange +(63) 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, [plan_id=2] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(64) HashAggregate [codegen id : 4] +(64) CometColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] + +(65) 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))#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] -(65) Filter [codegen id : 4] +(66) 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)))) -(66) CometScan parquet spark_catalog.default.store_sales +(67) CometScan 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 [] @@ -405,292 +410,299 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(67) CometFilter +(68) 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) -(68) ReusedExchange [Reuses operator id: 46] +(69) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#65] -(69) CometBroadcastHashJoin +(70) 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 -(70) ReusedExchange [Reuses operator id: 52] +(71) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(71) CometBroadcastHashJoin +(72) 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 -(72) CometProject +(73) 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] -(73) CometScan parquet spark_catalog.default.date_dim +(74) CometScan 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 -(74) CometFilter +(75) 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)) -(75) CometProject +(76) CometProject Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(76) CometBroadcastExchange +(77) CometBroadcastExchange Input [1]: [d_date_sk#70] Arguments: [d_date_sk#70] -(77) CometBroadcastHashJoin +(78) 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 -(78) CometProject +(79) 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] -(79) ColumnarToRow [codegen id : 2] +(80) CometColumnarToRow [codegen id : 2] Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] -(80) HashAggregate [codegen id : 2] +(81) 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] -(81) Exchange +(82) 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] + +(83) CometColumnarToRow [codegen id : 3] 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, [plan_id=3] -(82) HashAggregate [codegen id : 3] +(84) 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] -(83) Filter [codegen id : 3] +(85) 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)))) -(84) BroadcastExchange +(86) 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] -(85) BroadcastHashJoin [codegen id : 4] +(87) 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#67, i_class_id#68, i_category_id#69] Join type: Inner Join condition: None -(86) TakeOrderedAndProject +(88) 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 = 65 Hosting Expression = Subquery scalar-subquery#58, [id=#59] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- * ColumnarToRow (100) - +- CometUnion (99) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometScan parquet spark_catalog.default.store_sales (87) - : +- ReusedExchange (88) - :- CometProject (94) - : +- CometBroadcastHashJoin (93) - : :- CometScan parquet spark_catalog.default.catalog_sales (91) - : +- ReusedExchange (92) - +- CometProject (98) - +- CometBroadcastHashJoin (97) - :- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (96) - - -(87) CometScan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#58, [id=#59] +* HashAggregate (106) ++- * CometColumnarToRow (105) + +- CometColumnarExchange (104) + +- * HashAggregate (103) + +- * CometColumnarToRow (102) + +- CometUnion (101) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.store_sales (89) + : +- ReusedExchange (90) + :- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometScan parquet spark_catalog.default.catalog_sales (93) + : +- ReusedExchange (94) + +- CometProject (100) + +- CometBroadcastHashJoin (99) + :- CometScan parquet spark_catalog.default.web_sales (97) + +- ReusedExchange (98) + + +(89) CometScan 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#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(88) ReusedExchange [Reuses operator id: 19] +(90) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#89] -(89) CometBroadcastHashJoin +(91) 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 -(90) CometProject +(92) 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] -(91) CometScan parquet spark_catalog.default.catalog_sales +(93) CometScan 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#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(92) ReusedExchange [Reuses operator id: 19] +(94) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#96] -(93) CometBroadcastHashJoin +(95) 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 -(94) CometProject +(96) 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] -(95) CometScan parquet spark_catalog.default.web_sales +(97) CometScan 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#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#102)] ReadSchema: struct -(96) ReusedExchange [Reuses operator id: 19] +(98) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#103] -(97) CometBroadcastHashJoin +(99) 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 -(98) CometProject +(100) 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] -(99) CometUnion +(101) 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] -(100) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#90, list_price#91] -(101) HashAggregate [codegen id : 1] +(103) HashAggregate [codegen id : 1] Input [2]: [quantity#90, list_price#91] Keys: [] 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] -(102) Exchange +(104) CometColumnarExchange +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(105) CometColumnarToRow [codegen id : 2] Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(103) HashAggregate [codegen id : 2] +(106) HashAggregate [codegen id : 2] Input [2]: [sum#108, count#109] Keys: [] 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 = 87 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 97 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 (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (111) ++- * CometColumnarToRow (110) + +- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(104) CometScan parquet spark_catalog.default.date_dim +(107) CometScan 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 -(105) CometFilter +(108) 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)) -(106) CometProject +(109) CometProject Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] +(110) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(108) BroadcastExchange +(111) BroadcastExchange Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:7 Hosting operator id = 107 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* CometColumnarToRow (115) ++- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(109) CometScan parquet spark_catalog.default.date_dim +(112) CometScan 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 -(110) CometFilter +(113) 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)) -(111) CometProject +(114) 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] -(112) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#112] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (120) ++- * CometColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(113) CometScan parquet spark_catalog.default.date_dim +(116) CometScan 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 -(114) CometFilter +(117) 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)) -(115) CometProject +(118) CometProject Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(117) BroadcastExchange +(120) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] @@ -702,67 +714,67 @@ Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] +Subquery:13 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] -Subquery:14 Hosting operator id = 66 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:14 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 +BroadcastExchange (125) ++- * CometColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(118) CometScan parquet spark_catalog.default.date_dim +(121) CometScan 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 -(119) CometFilter +(122) 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)) -(120) CometProject +(123) CometProject Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(121) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#70] -(122) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#70] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:15 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:15 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:16 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:16 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +* CometColumnarToRow (129) ++- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(123) CometScan parquet spark_catalog.default.date_dim +(126) CometScan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct -(124) CometFilter +(127) CometFilter Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d_dom#120)) AND (d_year#118 = 1999)) AND (d_moy#119 = 12)) AND (d_dom#120 = 11)) -(125) CometProject +(128) CometProject Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] +(129) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] -Subquery:17 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:17 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:18 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:18 Hosting operator id = 74 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 1fecc0fca4..5ac344b17d 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 @@ -5,163 +5,166 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Subquery #4 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] #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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + CometColumnarToRow + 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 (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 #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] - 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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (1) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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] + 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] + CometExchange [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) + CometColumnarToRow + 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] - 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] + 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] + ReusedSubquery [d_week_seq] #2 InputAdapter 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] #16 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (2) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 fe3c7a681f..c8f5a94811 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,28 @@ == 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 (24) ++- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometColumnarExchange (21) + +- * HashAggregate (20) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -112,7 +113,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#10], Inner, BuildRight Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10] Arguments: [cs_sales_price#2, ca_zip#9], [cs_sales_price#2, ca_zip#9] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [2]: [cs_sales_price#2, ca_zip#9] (20) HashAggregate [codegen id : 1] @@ -122,50 +123,53 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Aggregate Attributes [1]: [sum#13] Results [2]: [ca_zip#9, sum#14] -(21) Exchange +(21) CometColumnarExchange Input [2]: [ca_zip#9, sum#14] -Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [codegen id : 2] +Input [2]: [ca_zip#9, sum#14] + +(23) 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 +(24) 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(24) CometScan parquet spark_catalog.default.date_dim +(25) CometScan 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 +(26) 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(28) BroadcastExchange +(29) 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 a34cd9c1b3..0e1ced3e2b 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,36 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_zip] #1 + WholeStageCodegen (1) + HashAggregate [ca_zip,cs_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 1ff243ff2a..81e5d013e1 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * HashAggregate (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.call_center (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.call_center (29) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -198,7 +199,7 @@ Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight 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] Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (36) HashAggregate [codegen id : 1] @@ -222,11 +223,14 @@ Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(Unscaled 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] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#22, sum#23, count#25] + +(41) HashAggregate [codegen id : 2] 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 72df9cecef..a0d8afa1dc 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - 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] - CometExchange [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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [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] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] + CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [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_order_number,cs_warehouse_sk] + 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] + CometExchange [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] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [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] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + 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 dab3dfbf36..94cae1ae09 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,47 @@ == 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 (43) ++- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -221,7 +222,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#26], Inner, BuildRight Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_sk#26, i_item_id#27, i_item_desc#28] Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#25, i_item_id#27, i_item_desc#28] (39) HashAggregate [codegen id : 1] @@ -231,80 +232,83 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#27, i_item_desc#28, s_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(41) HashAggregate [codegen id : 2] +(41) CometColumnarToRow [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] + +(42) 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 +(43) 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 (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan 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 +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(48) 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 (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(48) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 +(50) 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 +(51) 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] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] -(52) BroadcastExchange +(53) 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 4a634b0699..e362f97250 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,64 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow 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] - 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 [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) + CometColumnarToRow + 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] + 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 414a066f7c..1239f65147 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,46 @@ == 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 (42) ++- * HashAggregate (41) + +- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -214,7 +215,7 @@ Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, c Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22] Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#30, ca_state#31, ca_county#32, spark_grouping_id#33] (38) HashAggregate [codegen id : 1] @@ -224,50 +225,53 @@ 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) 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, [plan_id=1] +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] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [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] + +(41) 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 +(42) 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 (47) ++- * CometColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(42) CometScan parquet spark_catalog.default.date_dim +(43) CometScan 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 +(44) 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 +(45) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(46) BroadcastExchange +(47) 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 c49bfc6c88..2663509529 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,54 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 69d887880b..b890d91571 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,39 @@ == 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 (35) ++- * HashAggregate (34) + +- * CometColumnarToRow (33) + +- CometColumnarExchange (32) + +- * HashAggregate (31) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -175,7 +176,7 @@ Arguments: [ss_store_sk#6], [s_store_sk#19], Inner, NOT (substr(ca_zip#18, 1, 5) Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20] Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] (31) HashAggregate [codegen id : 1] @@ -185,18 +186,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarToRow [codegen id : 2] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] + +(34) 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 +(35) 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 60caecf91e..bda560adfb 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,39 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 17f54d24ad..350db52a99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt @@ -1,40 +1,42 @@ == Physical Plan == -* ColumnarToRow (36) -+- CometSort (35) - +- CometColumnarExchange (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) +* CometColumnarToRow (38) ++- CometSort (37) + +- CometColumnarExchange (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * HashAggregate (15) + : : +- * CometColumnarToRow (14) + : : +- CometColumnarExchange (13) + : : +- * HashAggregate (12) + : : +- * CometColumnarToRow (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 (20) + : +- * CometColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- BroadcastExchange (33) + +- * Project (32) + +- * BroadcastHashJoin Inner BuildRight (31) + :- * HashAggregate (25) + : +- * CometColumnarToRow (24) + : +- ReusedExchange (23) + +- BroadcastExchange (30) + +- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) (1) CometScan parquet spark_catalog.default.web_sales @@ -87,7 +89,7 @@ Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] Arguments: [sales_price#4, d_week_seq#10, d_day_name#11], [sales_price#4, d_week_seq#10, d_day_name#11] -(11) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [codegen id : 1] Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] (12) HashAggregate [codegen id : 1] @@ -97,113 +99,119 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(14) HashAggregate [codegen id : 6] +(14) CometColumnarToRow [codegen id : 6] +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] + +(15) 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) CometScan parquet spark_catalog.default.date_dim +(16) CometScan 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 +(17) CometFilter Input [2]: [d_week_seq#40, d_year#41] Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) -(17) CometProject +(18) CometProject Input [2]: [d_week_seq#40, d_year#41] Arguments: [d_week_seq#40], [d_week_seq#40] -(18) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [1]: [d_week_seq#40] -(19) BroadcastExchange +(20) 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] +(21) 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] +(22) 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] +(23) ReusedExchange [Reuses operator id: 13] Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] -(23) HashAggregate [codegen id : 5] +(24) CometColumnarToRow [codegen id : 5] +Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] + +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometProject Input [2]: [d_week_seq#67, d_year#68] Arguments: [d_week_seq#67], [d_week_seq#67] -(27) ColumnarToRow [codegen id : 4] +(29) CometColumnarToRow [codegen id : 4] Input [1]: [d_week_seq#67] -(28) BroadcastExchange +(30) 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] +(31) 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] +(32) 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 +(33) 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] +(34) 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] +(35) 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) CometColumnarExchange +(36) 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] -(35) CometSort +(37) 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: [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] -(36) ColumnarToRow [codegen id : 7] +(38) CometColumnarToRow [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] 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 6991754b7e..648b0e5b74 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,5 +1,5 @@ WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -9,26 +9,27 @@ WholeStageCodegen (7) 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] - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year] @@ -39,12 +40,13 @@ WholeStageCodegen (7) 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 + CometColumnarToRow + InputAdapter + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter BroadcastExchange #6 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [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 9eb32de680..be245743bf 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,28 @@ == 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 (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -87,7 +89,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -97,66 +99,72 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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 +(24) 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) CometScan parquet spark_catalog.default.date_dim +(25) CometScan 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 +(26) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject +(27) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=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 5bfd9a907c..6e0b46f7db 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,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 16ce2c766f..ffa1ecae34 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,29 @@ == 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 (25) ++- * Filter (24) + +- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometColumnarExchange (21) + +- * HashAggregate (20) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.inventory @@ -113,7 +114,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#11], Inner, BuildRight Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12] Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] (20) HashAggregate [codegen id : 1] @@ -123,49 +124,52 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [codegen id : 2] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] + +(23) 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] +(24) 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 (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#20 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#19 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#20 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#19 as double)))))) <= 1.5) END) -(24) TakeOrderedAndProject +(25) 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 (29) ++- * CometColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(25) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] -(28) BroadcastExchange +(29) 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 2c8892b689..0c089ac131 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,35 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 21c5568c2b..fa91be6f49 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,29 @@ == 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 (25) ++- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.inventory @@ -117,7 +118,7 @@ Arguments: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_ Input [5]: [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11] Arguments: [[inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, i_category#11, 0], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, i_class#10, null, 1], [inv_quantity_on_hand#3, i_product_name#12, i_brand#9, null, null, 3], [inv_quantity_on_hand#3, i_product_name#12, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [6]: [inv_quantity_on_hand#3, i_product_name#14, i_brand#15, i_class#16, i_category#17, spark_grouping_id#18] (21) HashAggregate [codegen id : 1] @@ -127,50 +128,53 @@ 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) 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, [plan_id=1] +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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarToRow [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] + +(24) 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 +(25) 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 (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(25) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) 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] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(29) BroadcastExchange +(30) 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 eb25d6d3ee..bc70715439 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,37 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 00a790d59a..303a64bf12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt @@ -1,70 +1,73 @@ == Physical Plan == -* HashAggregate (66) -+- Exchange (65) - +- * HashAggregate (64) - +- Union (63) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) - : :- * Project (42) - : : +- * SortMergeJoin LeftSemi (41) - : : :- * ColumnarToRow (25) - : : : +- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (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 (40) - : : +- * Project (39) - : : +- * Filter (38) - : : +- * HashAggregate (37) - : : +- Exchange (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 (43) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * SortMergeJoin LeftSemi (58) - : :- * ColumnarToRow (52) - : : +- CometSort (51) - : : +- CometExchange (50) - : : +- CometProject (49) - : : +- CometBroadcastHashJoin (48) - : : :- CometScan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (47) - : +- * Sort (57) - : +- * Project (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) +* HashAggregate (69) ++- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- Union (65) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * SortMergeJoin LeftSemi (42) + : : :- * CometColumnarToRow (25) + : : : +- CometSort (24) + : : : +- CometExchange (23) + : : : +- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometBroadcastExchange (20) + : : : +- CometProject (19) + : : : +- CometFilter (18) + : : : +- CometHashAggregate (17) + : : : +- CometExchange (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 (41) + : : +- * Project (40) + : : +- * Filter (39) + : : +- * HashAggregate (38) + : : +- * CometColumnarToRow (37) + : : +- CometColumnarExchange (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (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 (44) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * SortMergeJoin LeftSemi (60) + : :- * CometColumnarToRow (53) + : : +- CometSort (52) + : : +- CometExchange (51) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometScan parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (48) + : +- * Sort (59) + : +- * Project (58) + : +- * Filter (57) + : +- * HashAggregate (56) + : +- * CometColumnarToRow (55) + : +- ReusedExchange (54) + +- ReusedExchange (62) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +184,7 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come 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, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (26) CometScan parquet spark_catalog.default.store_sales @@ -223,7 +226,7 @@ Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight 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] +(34) CometColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] (35) HashAggregate [codegen id : 2] @@ -233,141 +236,150 @@ Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_pr Aggregate Attributes [2]: [sum#24, isEmpty#25] Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(36) Exchange +(36) CometColumnarExchange Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(37) HashAggregate [codegen id : 3] +(37) CometColumnarToRow [codegen id : 3] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(38) 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] -(38) Filter [codegen id : 3] +(39) 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]))) -(39) Project [codegen id : 3] +(40) Project [codegen id : 3] Output [1]: [c_customer_sk#23] Input [2]: [c_customer_sk#23, ssales#29] -(40) Sort [codegen id : 3] +(41) Sort [codegen id : 3] Input [1]: [c_customer_sk#23] Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 -(41) SortMergeJoin [codegen id : 5] +(42) SortMergeJoin [codegen id : 5] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(42) Project [codegen id : 5] +(43) 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] -(43) ReusedExchange [Reuses operator id: 71] +(44) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#32] -(44) BroadcastHashJoin [codegen id : 5] +(45) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#32] Join type: Inner Join condition: None -(45) Project [codegen id : 5] +(46) 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] -(46) CometScan parquet spark_catalog.default.web_sales +(47) CometScan 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#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] ReadSchema: struct -(47) ReusedExchange [Reuses operator id: 20] +(48) ReusedExchange [Reuses operator id: 20] Output [1]: [item_sk#40] -(48) CometBroadcastHashJoin +(49) 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 -(49) CometProject +(50) 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] -(50) CometExchange +(51) CometExchange 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, CometNativeShuffle, [plan_id=4] -(51) CometSort +(52) 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) ColumnarToRow [codegen id : 6] +(53) CometColumnarToRow [codegen id : 6] Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -(53) ReusedExchange [Reuses operator id: 36] +(54) ReusedExchange [Reuses operator id: 36] Output [3]: [c_customer_sk#41, sum#42, isEmpty#43] -(54) HashAggregate [codegen id : 8] +(55) CometColumnarToRow [codegen id : 8] +Input [3]: [c_customer_sk#41, sum#42, isEmpty#43] + +(56) 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] -(55) Filter [codegen id : 8] +(57) 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]))) -(56) Project [codegen id : 8] +(58) Project [codegen id : 8] Output [1]: [c_customer_sk#41] Input [2]: [c_customer_sk#41, ssales#46] -(57) Sort [codegen id : 8] +(59) Sort [codegen id : 8] Input [1]: [c_customer_sk#41] Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 -(58) SortMergeJoin [codegen id : 10] +(60) 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 -(59) Project [codegen id : 10] +(61) 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] -(60) ReusedExchange [Reuses operator id: 71] +(62) ReusedExchange [Reuses operator id: 74] Output [1]: [d_date_sk#47] -(61) BroadcastHashJoin [codegen id : 10] +(63) 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 -(62) Project [codegen id : 10] +(64) 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] -(63) Union +(65) Union -(64) HashAggregate [codegen id : 11] +(66) HashAggregate [codegen id : 11] Input [1]: [sales#33] Keys: [] Functions [1]: [partial_sum(sales#33)] Aggregate Attributes [2]: [sum#49, isEmpty#50] Results [2]: [sum#51, isEmpty#52] -(65) Exchange +(67) CometColumnarExchange Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(66) HashAggregate [codegen id : 12] +(68) CometColumnarToRow [codegen id : 12] +Input [2]: [sum#51, isEmpty#52] + +(69) HashAggregate [codegen id : 12] Input [2]: [sum#51, isEmpty#52] Keys: [] Functions [1]: [sum(sales#33)] @@ -377,87 +389,89 @@ 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 (71) -+- * ColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +BroadcastExchange (74) ++- * CometColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan parquet spark_catalog.default.date_dim (70) -(67) CometScan parquet spark_catalog.default.date_dim +(70) CometScan 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 -(68) CometFilter +(71) 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)) -(69) CometProject +(72) CometProject Input [3]: [d_date_sk#32, d_year#55, d_moy#56] Arguments: [d_date_sk#32], [d_date_sk#32] -(70) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] -(71) BroadcastExchange +(74) 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 = 2 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 (79) ++- * CometColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) -(72) CometScan parquet spark_catalog.default.date_dim +(75) CometScan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(76) CometFilter Input [3]: [d_date_sk#10, d_date#11, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(74) CometProject +(77) CometProject Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(75) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(76) BroadcastExchange +(79) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* HashAggregate (94) -+- Exchange (93) - +- * HashAggregate (92) - +- * HashAggregate (91) - +- Exchange (90) - +- * HashAggregate (89) - +- * ColumnarToRow (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometFilter (78) - : : +- CometScan parquet spark_catalog.default.store_sales (77) - : +- ReusedExchange (79) - +- CometBroadcastExchange (85) - +- CometProject (84) - +- CometFilter (83) - +- CometScan parquet spark_catalog.default.date_dim (82) - - -(77) CometScan parquet spark_catalog.default.store_sales +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (99) ++- * CometColumnarToRow (98) + +- CometColumnarExchange (97) + +- * HashAggregate (96) + +- * HashAggregate (95) + +- * CometColumnarToRow (94) + +- CometColumnarExchange (93) + +- * HashAggregate (92) + +- * CometColumnarToRow (91) + +- CometProject (90) + +- CometBroadcastHashJoin (89) + :- CometProject (84) + : +- CometBroadcastHashJoin (83) + : :- CometFilter (81) + : : +- CometScan parquet spark_catalog.default.store_sales (80) + : +- ReusedExchange (82) + +- CometBroadcastExchange (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan parquet spark_catalog.default.date_dim (85) + + +(80) CometScan 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 [] @@ -465,121 +479,127 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#60), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(78) CometFilter +(81) 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) -(79) ReusedExchange [Reuses operator id: 31] +(82) ReusedExchange [Reuses operator id: 31] Output [1]: [c_customer_sk#62] -(80) CometBroadcastHashJoin +(83) 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 -(81) CometProject +(84) 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] -(82) CometScan parquet spark_catalog.default.date_dim +(85) CometScan 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 -(83) CometFilter +(86) 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)) -(84) CometProject +(87) CometProject Input [2]: [d_date_sk#63, d_year#64] Arguments: [d_date_sk#63], [d_date_sk#63] -(85) CometBroadcastExchange +(88) CometBroadcastExchange Input [1]: [d_date_sk#63] Arguments: [d_date_sk#63] -(86) CometBroadcastHashJoin +(89) 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 -(87) CometProject +(90) 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] -(88) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#58, ss_sales_price#59, c_customer_sk#62] -(89) HashAggregate [codegen id : 1] +(92) 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] -(90) Exchange +(93) CometColumnarExchange Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(c_customer_sk#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(91) HashAggregate [codegen id : 2] +(94) CometColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#62, sum#67, isEmpty#68] + +(95) 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] -(92) HashAggregate [codegen id : 2] +(96) HashAggregate [codegen id : 2] Input [1]: [csales#70] Keys: [] Functions [1]: [partial_max(csales#70)] Aggregate Attributes [1]: [max#71] Results [1]: [max#72] -(93) Exchange +(97) CometColumnarExchange +Input [1]: [max#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(98) CometColumnarToRow [codegen id : 3] Input [1]: [max#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(94) HashAggregate [codegen id : 3] +(99) HashAggregate [codegen id : 3] Input [1]: [max#72] Keys: [] 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 = 77 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 -BroadcastExchange (99) -+- * ColumnarToRow (98) - +- CometProject (97) - +- CometFilter (96) - +- CometScan parquet spark_catalog.default.date_dim (95) +Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#60 IN dynamicpruning#61 +BroadcastExchange (104) ++- * CometColumnarToRow (103) + +- CometProject (102) + +- CometFilter (101) + +- CometScan parquet spark_catalog.default.date_dim (100) -(95) CometScan parquet spark_catalog.default.date_dim +(100) CometScan 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 -(96) CometFilter +(101) 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)) -(97) CometProject +(102) CometProject Input [2]: [d_date_sk#63, d_year#64] Arguments: [d_date_sk#63], [d_date_sk#63] -(98) ColumnarToRow [codegen id : 1] +(103) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#63] -(99) BroadcastExchange +(104) 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 = 46 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] +Subquery:6 Hosting operator id = 57 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 760cbdd8bc..4b1d62c06f 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,139 +1,144 @@ WholeStageCodegen (12) HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] - InputAdapter - Exchange #1 - 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] - CometExchange [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)] - CometExchange [_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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + 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) + CometColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometExchange [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) + CometColumnarToRow + 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)] + CometExchange [_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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #12 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #11 + CometColumnarExchange [c_customer_sk] #9 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] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + CometColumnarToRow 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 (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] - CometExchange [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 - 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] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 + 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) + CometColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometExchange [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 + 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] + CometColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #9 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt index 56684d343b..6cb57a00a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt @@ -1,91 +1,96 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- Union (86) - :- * HashAggregate (62) - : +- Exchange (61) - : +- * HashAggregate (60) - : +- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * SortMergeJoin LeftSemi (42) - : : : :- * ColumnarToRow (26) - : : : : +- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (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) - : : +- BroadcastExchange (54) - : : +- * SortMergeJoin LeftSemi (53) - : : :- * ColumnarToRow (47) - : : : +- CometSort (46) - : : : +- CometExchange (45) - : : : +- CometFilter (44) - : : : +- CometScan parquet spark_catalog.default.customer (43) - : : +- * Sort (52) - : : +- * Project (51) - : : +- * Filter (50) - : : +- * HashAggregate (49) - : : +- ReusedExchange (48) - : +- ReusedExchange (57) - +- * HashAggregate (85) - +- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * BroadcastHashJoin Inner BuildRight (81) - :- * Project (79) - : +- * BroadcastHashJoin Inner BuildRight (78) - : :- * SortMergeJoin LeftSemi (76) - : : :- * ColumnarToRow (70) - : : : +- CometSort (69) - : : : +- CometExchange (68) - : : : +- CometProject (67) - : : : +- CometBroadcastHashJoin (66) - : : : :- CometFilter (64) - : : : : +- CometScan parquet spark_catalog.default.web_sales (63) - : : : +- ReusedExchange (65) - : : +- * Sort (75) - : : +- * Project (74) - : : +- * Filter (73) - : : +- * HashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) +TakeOrderedAndProject (92) ++- Union (91) + :- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (58) + : : +- * BroadcastHashJoin Inner BuildRight (57) + : : :- * SortMergeJoin LeftSemi (43) + : : : :- * CometColumnarToRow (26) + : : : : +- CometSort (25) + : : : : +- CometExchange (24) + : : : : +- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometHashAggregate (18) + : : : : +- CometExchange (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 (42) + : : : +- * Project (41) + : : : +- * Filter (40) + : : : +- * HashAggregate (39) + : : : +- * CometColumnarToRow (38) + : : : +- CometColumnarExchange (37) + : : : +- * HashAggregate (36) + : : : +- * CometColumnarToRow (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 (56) + : : +- * SortMergeJoin LeftSemi (55) + : : :- * CometColumnarToRow (48) + : : : +- CometSort (47) + : : : +- CometExchange (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.customer (44) + : : +- * Sort (54) + : : +- * Project (53) + : : +- * Filter (52) + : : +- * HashAggregate (51) + : : +- * CometColumnarToRow (50) + : : +- ReusedExchange (49) + : +- ReusedExchange (59) + +- * HashAggregate (90) + +- * CometColumnarToRow (89) + +- CometColumnarExchange (88) + +- * HashAggregate (87) + +- * Project (86) + +- * BroadcastHashJoin Inner BuildRight (85) + :- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * SortMergeJoin LeftSemi (80) + : : :- * CometColumnarToRow (73) + : : : +- CometSort (72) + : : : +- CometExchange (71) + : : : +- CometProject (70) + : : : +- CometBroadcastHashJoin (69) + : : : :- CometFilter (67) + : : : : +- CometScan parquet spark_catalog.default.web_sales (66) + : : : +- ReusedExchange (68) + : : +- * Sort (79) + : : +- * Project (78) + : : +- * Filter (77) + : : +- * HashAggregate (76) + : : +- * CometColumnarToRow (75) + : : +- ReusedExchange (74) + : +- ReusedExchange (81) + +- ReusedExchange (84) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -207,7 +212,7 @@ Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, Come 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, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (27) CometScan parquet spark_catalog.default.store_sales @@ -249,7 +254,7 @@ Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight 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] +(35) CometColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] (36) HashAggregate [codegen id : 2] @@ -259,131 +264,140 @@ Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_pr Aggregate Attributes [2]: [sum#24, isEmpty#25] Results [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(37) Exchange +(37) CometColumnarExchange Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] -(39) Filter [codegen id : 3] +(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]))) -(40) Project [codegen id : 3] +(41) Project [codegen id : 3] Output [1]: [c_customer_sk#23] Input [2]: [c_customer_sk#23, ssales#29] -(41) Sort [codegen id : 3] +(42) Sort [codegen id : 3] Input [1]: [c_customer_sk#23] Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 -(42) SortMergeJoin [codegen id : 9] +(43) SortMergeJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(43) CometScan parquet spark_catalog.default.customer +(44) CometScan 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 -(44) CometFilter +(45) CometFilter Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] Condition : isnotnull(c_customer_sk#32) -(45) CometExchange +(46) CometExchange Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] Arguments: hashpartitioning(c_customer_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(46) CometSort +(47) 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] -(47) ColumnarToRow [codegen id : 4] +(48) CometColumnarToRow [codegen id : 4] Input [3]: [c_customer_sk#32, c_first_name#33, c_last_name#34] -(48) ReusedExchange [Reuses operator id: 37] +(49) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#23, sum#26, isEmpty#27] -(49) HashAggregate [codegen id : 6] +(50) CometColumnarToRow [codegen id : 6] +Input [3]: [c_customer_sk#23, sum#26, isEmpty#27] + +(51) 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] -(50) Filter [codegen id : 6] +(52) 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]))) -(51) Project [codegen id : 6] +(53) Project [codegen id : 6] Output [1]: [c_customer_sk#23] Input [2]: [c_customer_sk#23, ssales#29] -(52) Sort [codegen id : 6] +(54) Sort [codegen id : 6] Input [1]: [c_customer_sk#23] Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 7] +(55) SortMergeJoin [codegen id : 7] Left keys [1]: [c_customer_sk#32] Right keys [1]: [c_customer_sk#23] Join type: LeftSemi Join condition: None -(54) BroadcastExchange +(56) 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] -(55) BroadcastHashJoin [codegen id : 9] +(57) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_bill_customer_sk#1] Right keys [1]: [c_customer_sk#32] Join type: Inner Join condition: None -(56) Project [codegen id : 9] +(58) 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] -(57) ReusedExchange [Reuses operator id: 92] +(59) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#35] -(58) BroadcastHashJoin [codegen id : 9] +(60) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#5] Right keys [1]: [d_date_sk#35] Join type: Inner Join condition: None -(59) Project [codegen id : 9] +(61) 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] -(60) HashAggregate [codegen id : 9] +(62) 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#36, isEmpty#37] Results [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -(61) Exchange +(63) 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, [plan_id=6] +Arguments: hashpartitioning(c_last_name#34, c_first_name#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(62) HashAggregate [codegen id : 10] +(64) CometColumnarToRow [codegen id : 10] +Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] + +(65) 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))#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] -(63) CometScan parquet spark_catalog.default.web_sales +(66) CometScan 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 [] @@ -391,195 +405,203 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(64) CometFilter +(67) 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) -(65) ReusedExchange [Reuses operator id: 21] +(68) ReusedExchange [Reuses operator id: 21] Output [1]: [item_sk#48] -(66) CometBroadcastHashJoin +(69) 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 -(67) CometProject +(70) 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] -(68) CometExchange +(71) CometExchange 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, CometNativeShuffle, [plan_id=7] -(69) CometSort +(72) 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) ColumnarToRow [codegen id : 11] +(73) CometColumnarToRow [codegen id : 11] Input [4]: [ws_bill_customer_sk#43, ws_quantity#44, ws_list_price#45, ws_sold_date_sk#46] -(71) ReusedExchange [Reuses operator id: 37] +(74) ReusedExchange [Reuses operator id: 37] Output [3]: [c_customer_sk#49, sum#50, isEmpty#51] -(72) HashAggregate [codegen id : 13] +(75) CometColumnarToRow [codegen id : 13] +Input [3]: [c_customer_sk#49, sum#50, isEmpty#51] + +(76) 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] -(73) Filter [codegen id : 13] +(77) 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]))) -(74) Project [codegen id : 13] +(78) Project [codegen id : 13] Output [1]: [c_customer_sk#49] Input [2]: [c_customer_sk#49, ssales#54] -(75) Sort [codegen id : 13] +(79) Sort [codegen id : 13] Input [1]: [c_customer_sk#49] Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0 -(76) SortMergeJoin [codegen id : 19] +(80) 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 -(77) ReusedExchange [Reuses operator id: 54] +(81) ReusedExchange [Reuses operator id: 56] Output [3]: [c_customer_sk#55, c_first_name#56, c_last_name#57] -(78) BroadcastHashJoin [codegen id : 19] +(82) 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 -(79) Project [codegen id : 19] +(83) 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] -(80) ReusedExchange [Reuses operator id: 92] +(84) ReusedExchange [Reuses operator id: 97] Output [1]: [d_date_sk#58] -(81) BroadcastHashJoin [codegen id : 19] +(85) 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 -(82) Project [codegen id : 19] +(86) 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] -(83) HashAggregate [codegen id : 19] +(87) 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] -(84) Exchange +(88) 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, [plan_id=8] +Arguments: hashpartitioning(c_last_name#57, c_first_name#56, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(85) HashAggregate [codegen id : 20] +(89) CometColumnarToRow [codegen id : 20] +Input [4]: [c_last_name#57, c_first_name#56, sum#61, isEmpty#62] + +(90) 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] -(86) Union +(91) Union -(87) TakeOrderedAndProject +(92) 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 (92) -+- * ColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan parquet spark_catalog.default.date_dim (88) +BroadcastExchange (97) ++- * CometColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan parquet spark_catalog.default.date_dim (93) -(88) CometScan parquet spark_catalog.default.date_dim +(93) CometScan 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 -(89) CometFilter +(94) 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)) -(90) CometProject +(95) CometProject Input [3]: [d_date_sk#35, d_year#65, d_moy#66] Arguments: [d_date_sk#35], [d_date_sk#35] -(91) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#35] -(92) BroadcastExchange +(97) 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 = 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) +BroadcastExchange (102) ++- * CometColumnarToRow (101) + +- CometProject (100) + +- CometFilter (99) + +- CometScan parquet spark_catalog.default.date_dim (98) -(93) CometScan parquet spark_catalog.default.date_dim +(98) CometScan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_date#11, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter +(99) CometFilter Input [3]: [d_date_sk#10, d_date#11, d_year#12] Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) -(95) CometProject +(100) CometProject Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] -(97) BroadcastExchange +(102) BroadcastExchange Input [2]: [d_date_sk#10, d_date#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#30, [id=#31] -* HashAggregate (115) -+- Exchange (114) - +- * HashAggregate (113) - +- * HashAggregate (112) - +- Exchange (111) - +- * HashAggregate (110) - +- * ColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#30, [id=#31] +* HashAggregate (122) ++- * CometColumnarToRow (121) + +- CometColumnarExchange (120) + +- * HashAggregate (119) + +- * HashAggregate (118) + +- * CometColumnarToRow (117) + +- CometColumnarExchange (116) + +- * HashAggregate (115) + +- * CometColumnarToRow (114) + +- CometProject (113) + +- CometBroadcastHashJoin (112) + :- CometProject (107) + : +- CometBroadcastHashJoin (106) + : :- CometFilter (104) + : : +- CometScan parquet spark_catalog.default.store_sales (103) + : +- ReusedExchange (105) + +- CometBroadcastExchange (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) + + +(103) CometScan 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 [] @@ -587,123 +609,129 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(99) CometFilter +(104) 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) -(100) ReusedExchange [Reuses operator id: 32] +(105) ReusedExchange [Reuses operator id: 32] Output [1]: [c_customer_sk#72] -(101) CometBroadcastHashJoin +(106) 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 -(102) CometProject +(107) 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] -(103) CometScan parquet spark_catalog.default.date_dim +(108) CometScan 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 -(104) CometFilter +(109) 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)) -(105) CometProject +(110) CometProject Input [2]: [d_date_sk#73, d_year#74] Arguments: [d_date_sk#73], [d_date_sk#73] -(106) CometBroadcastExchange +(111) CometBroadcastExchange Input [1]: [d_date_sk#73] Arguments: [d_date_sk#73] -(107) CometBroadcastHashJoin +(112) 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 -(108) CometProject +(113) 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] -(109) ColumnarToRow [codegen id : 1] +(114) CometColumnarToRow [codegen id : 1] Input [3]: [ss_quantity#68, ss_sales_price#69, c_customer_sk#72] -(110) HashAggregate [codegen id : 1] +(115) 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] -(111) Exchange +(116) CometColumnarExchange Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] -Arguments: hashpartitioning(c_customer_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(c_customer_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(112) HashAggregate [codegen id : 2] +(117) CometColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#72, sum#77, isEmpty#78] + +(118) 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] -(113) HashAggregate [codegen id : 2] +(119) HashAggregate [codegen id : 2] Input [1]: [csales#80] Keys: [] Functions [1]: [partial_max(csales#80)] Aggregate Attributes [1]: [max#81] Results [1]: [max#82] -(114) Exchange +(120) CometColumnarExchange +Input [1]: [max#82] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(121) CometColumnarToRow [codegen id : 3] Input [1]: [max#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(115) HashAggregate [codegen id : 3] +(122) HashAggregate [codegen id : 3] Input [1]: [max#82] Keys: [] 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 = 98 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (120) -+- * ColumnarToRow (119) - +- CometProject (118) - +- CometFilter (117) - +- CometScan parquet spark_catalog.default.date_dim (116) +Subquery:4 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#71 +BroadcastExchange (127) ++- * CometColumnarToRow (126) + +- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) -(116) CometScan parquet spark_catalog.default.date_dim +(123) CometScan 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 -(117) CometFilter +(124) 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)) -(118) CometProject +(125) CometProject Input [2]: [d_date_sk#73, d_year#74] Arguments: [d_date_sk#73], [d_date_sk#73] -(119) ColumnarToRow [codegen id : 1] +(126) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#73] -(120) BroadcastExchange +(127) 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 = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] +Subquery:5 Hosting operator id = 52 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] -Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 66 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#30, [id=#31] +Subquery:7 Hosting operator id = 77 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 1184e50857..1ea0ed0d15 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 @@ -2,169 +2,176 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales] Union 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 (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] - CometExchange [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 - 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)] - CometExchange [_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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name] #1 + 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) + CometColumnarToRow + InputAdapter + CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + CometExchange [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) + CometColumnarToRow + 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)] + CometExchange [_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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #12 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #11 + CometColumnarExchange [c_customer_sk] #9 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] + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + CometColumnarToRow 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 (2) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - ColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [c_customer_sk,c_first_name,c_last_name] + CometExchange [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 (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] + CometColumnarToRow 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 - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [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 (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] - InputAdapter - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - InputAdapter - ReusedExchange [d_date_sk] #3 + 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 (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] - CometExchange [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 - ReusedExchange [item_sk] #4 + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name] #17 + 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) + CometColumnarToRow + InputAdapter + CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + CometExchange [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 + 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] + CometColumnarToRow + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #9 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] - 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 + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt index 9dbf4af839..0f0ae8e63d 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,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Condition : isnotnull(i_item_sk#51) -(54) CometBroadcastExchange +(56) CometBroadcastExchange Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] Right output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_item_sk#40], [i_item_sk#51], Inner, BuildRight -(56) CometProject +(58) CometProject Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56], [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Right output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight -(59) CometProject +(61) CometProject Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#61, ca_zip#62, ca_country#63] -(62) BroadcastHashJoin [codegen id : 2] +(64) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#60, s_zip#50] Right keys [2]: [upper(ca_country#63), ca_zip#62] Join type: Inner Join condition: None -(63) Project [codegen id : 2] +(65) Project [codegen id : 2] Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] -(64) HashAggregate [codegen id : 2] +(66) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum#64] Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -(65) Exchange +(67) CometColumnarExchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(68) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 3] +(69) HashAggregate [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#66] Keys: [] Functions [1]: [partial_avg(netpaid#66)] Aggregate Attributes [2]: [sum#67, count#68] Results [2]: [sum#69, count#70] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#69, count#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#69, count#70] Keys: [] Functions [1]: [avg(netpaid#66)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt index f27ae40199..c9756da952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#45, sr_ticket_number#46] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#45, sr_ticket_number#46] Arguments: [sr_item_sk#45, sr_ticket_number#46], [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Condition : isnotnull(i_item_sk#51) -(54) CometBroadcastExchange +(56) CometBroadcastExchange Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] Right output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_item_sk#40], [i_item_sk#51], Inner, BuildRight -(56) CometProject +(58) CometProject Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Arguments: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56], [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] Right output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_customer_sk#41], [c_customer_sk#57], Inner, BuildRight -(59) CometProject +(61) CometProject Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] Arguments: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60], [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [codegen id : 2] Input [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#61, ca_zip#62, ca_country#63] -(62) BroadcastHashJoin [codegen id : 2] +(64) BroadcastHashJoin [codegen id : 2] Left keys [2]: [c_birth_country#60, s_zip#50] Right keys [2]: [upper(ca_country#63), ca_zip#62] Join type: Inner Join condition: None -(63) Project [codegen id : 2] +(65) Project [codegen id : 2] Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] -(64) HashAggregate [codegen id : 2] +(66) HashAggregate [codegen id : 2] Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum#64] Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -(65) Exchange +(67) CometColumnarExchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(68) CometColumnarToRow [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] -Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=6] -(66) HashAggregate [codegen id : 3] +(69) HashAggregate [codegen id : 3] Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#66] Keys: [] Functions [1]: [partial_avg(netpaid#66)] Aggregate Attributes [2]: [sum#67, count#68] Results [2]: [sum#69, count#70] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#69, count#70] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#69, count#70] Keys: [] Functions [1]: [avg(netpaid#66)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt index 35ca891390..c5fb361820 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -1,46 +1,47 @@ == 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 (43) ++- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -221,7 +222,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#29], Inner, BuildRight Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_sk#29, i_item_id#30, i_item_desc#31] Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#27, s_store_name#28, i_item_id#30, i_item_desc#31] (39) HashAggregate [codegen id : 1] @@ -231,80 +232,83 @@ Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(Unscale Aggregate Attributes [3]: [sum#32, sum#33, sum#34] Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] -(40) Exchange +(40) CometColumnarExchange Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(41) HashAggregate [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] +Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] + +(42) HashAggregate [codegen id : 2] Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, sum#35, sum#36, sum#37] Keys [4]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#38, sum(UnscaledValue(sr_net_loss#11))#39, sum(UnscaledValue(cs_net_profit#16))#40] Results [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#38,17,2) AS store_sales_profit#41, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#39,17,2) AS store_returns_loss#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#40,17,2) AS catalog_sales_profit#43] -(42) TakeOrderedAndProject +(43) TakeOrderedAndProject Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] Arguments: 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#19, d_year#20, d_moy#21] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(47) BroadcastExchange +(48) 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 (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(48) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 +(50) 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 +(51) 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] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(52) BroadcastExchange +(53) 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 6335e50ff8..23c7891f8a 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,64 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow 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] - 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] + 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) + CometColumnarToRow + 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] + 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 0cbbe17335..361d409610 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,35 @@ == 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 (31) ++- * HashAggregate (30) + +- * CometColumnarToRow (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -151,7 +152,7 @@ Arguments: [cs_promo_sk#3], [p_promo_sk#18], Inner, BuildRight Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] (27) HashAggregate [codegen id : 1] @@ -161,50 +162,53 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(29) HashAggregate [codegen id : 2] +(29) CometColumnarToRow [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] + +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(36) 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 b98e61704b..010ea4c1b3 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,43 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 f136cc4562..d0601d53ab 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,35 @@ == 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 (31) ++- * HashAggregate (30) + +- * CometColumnarToRow (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -151,7 +152,7 @@ Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17] Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#21, spark_grouping_id#22] (27) HashAggregate [codegen id : 1] @@ -161,50 +162,53 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#20, s_state#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(29) HashAggregate [codegen id : 2] +(29) CometColumnarToRow [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] + +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(36) 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 4d4c824bc9..eed2b8bd15 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,43 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 c59c9e2309..4b4b5a6286 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,86 @@ == 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 (82) +:- * BroadcastNestedLoopJoin Inner BuildRight (68) +: :- * BroadcastNestedLoopJoin Inner BuildRight (54) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (40) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (26) +: : : : :- * HashAggregate (12) +: : : : : +- * CometColumnarToRow (11) +: : : : : +- CometColumnarExchange (10) +: : : : : +- * HashAggregate (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * CometColumnarToRow (7) +: : : : : +- CometColumnarExchange (6) +: : : : : +- * HashAggregate (5) +: : : : : +- * CometColumnarToRow (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (25) +: : : : +- * HashAggregate (24) +: : : : +- * CometColumnarToRow (23) +: : : : +- CometColumnarExchange (22) +: : : : +- * HashAggregate (21) +: : : : +- * HashAggregate (20) +: : : : +- * CometColumnarToRow (19) +: : : : +- CometColumnarExchange (18) +: : : : +- * HashAggregate (17) +: : : : +- * CometColumnarToRow (16) +: : : : +- CometProject (15) +: : : : +- CometFilter (14) +: : : : +- CometScan parquet spark_catalog.default.store_sales (13) +: : : +- BroadcastExchange (39) +: : : +- * HashAggregate (38) +: : : +- * CometColumnarToRow (37) +: : : +- CometColumnarExchange (36) +: : : +- * HashAggregate (35) +: : : +- * HashAggregate (34) +: : : +- * CometColumnarToRow (33) +: : : +- CometColumnarExchange (32) +: : : +- * HashAggregate (31) +: : : +- * CometColumnarToRow (30) +: : : +- CometProject (29) +: : : +- CometFilter (28) +: : : +- CometScan parquet spark_catalog.default.store_sales (27) +: : +- BroadcastExchange (53) +: : +- * HashAggregate (52) +: : +- * CometColumnarToRow (51) +: : +- CometColumnarExchange (50) +: : +- * HashAggregate (49) +: : +- * HashAggregate (48) +: : +- * CometColumnarToRow (47) +: : +- CometColumnarExchange (46) +: : +- * HashAggregate (45) +: : +- * CometColumnarToRow (44) +: : +- CometProject (43) +: : +- CometFilter (42) +: : +- CometScan parquet spark_catalog.default.store_sales (41) +: +- BroadcastExchange (67) +: +- * HashAggregate (66) +: +- * CometColumnarToRow (65) +: +- CometColumnarExchange (64) +: +- * HashAggregate (63) +: +- * HashAggregate (62) +: +- * CometColumnarToRow (61) +: +- CometColumnarExchange (60) +: +- * HashAggregate (59) +: +- * CometColumnarToRow (58) +: +- CometProject (57) +: +- CometFilter (56) +: +- CometScan parquet spark_catalog.default.store_sales (55) ++- BroadcastExchange (81) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometColumnarExchange (78) + +- * HashAggregate (77) + +- * HashAggregate (76) + +- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * CometColumnarToRow (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.store_sales (69) (1) CometScan parquet spark_catalog.default.store_sales @@ -86,7 +98,7 @@ Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quanti Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Arguments: [ss_list_price#3], [ss_list_price#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [1]: [ss_list_price#3] (5) HashAggregate [codegen id : 1] @@ -96,342 +108,378 @@ 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) CometColumnarExchange Input [4]: [ss_list_price#3, sum#8, count#9, count#10] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(7) HashAggregate [codegen id : 2] +(7) CometColumnarToRow [codegen id : 2] +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] + +(8) 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] +(9) 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 +(10) CometColumnarExchange Input [4]: [sum#8, count#9, count#10, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) HashAggregate [codegen id : 18] +(11) CometColumnarToRow [codegen id : 18] +Input [4]: [sum#8, count#9, count#10, count#12] + +(12) 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) CometScan parquet spark_catalog.default.store_sales +(13) CometScan 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 +(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) ColumnarToRow [codegen id : 3] +(16) CometColumnarToRow [codegen id : 3] Input [1]: [ss_list_price#18] -(15) HashAggregate [codegen id : 3] +(17) 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 +(18) CometColumnarExchange Input [4]: [ss_list_price#18, sum#23, count#24, count#25] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) HashAggregate [codegen id : 4] +(19) CometColumnarToRow [codegen id : 4] +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] + +(20) 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] +(21) 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 +(22) CometColumnarExchange +Input [4]: [sum#23, count#24, count#25, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(23) CometColumnarToRow [codegen id : 5] Input [4]: [sum#23, count#24, count#25, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(20) HashAggregate [codegen id : 5] +(24) 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 +(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 : 18] Join type: Inner Join condition: None -(23) CometScan parquet spark_catalog.default.store_sales +(27) CometScan 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) ColumnarToRow [codegen id : 6] +(30) CometColumnarToRow [codegen id : 6] Input [1]: [ss_list_price#33] -(27) HashAggregate [codegen id : 6] +(31) 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 +(32) 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] +(33) CometColumnarToRow [codegen id : 7] +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] + +(34) 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] +(35) 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 +(36) CometColumnarExchange +Input [4]: [sum#38, count#39, count#40, count#42] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(37) CometColumnarToRow [codegen id : 8] Input [4]: [sum#38, count#39, count#40, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(32) HashAggregate [codegen id : 8] +(38) 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 +(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 : 18] Join type: Inner Join condition: None -(35) CometScan parquet spark_catalog.default.store_sales +(41) CometScan 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) ColumnarToRow [codegen id : 9] +(44) CometColumnarToRow [codegen id : 9] Input [1]: [ss_list_price#48] -(39) HashAggregate [codegen id : 9] +(45) 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 +(46) 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] + +(47) CometColumnarToRow [codegen id : 10] 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] +(48) 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] +(49) 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 +(50) CometColumnarExchange Input [4]: [sum#53, count#54, count#55, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(44) HashAggregate [codegen id : 11] +(51) CometColumnarToRow [codegen id : 11] +Input [4]: [sum#53, count#54, count#55, count#57] + +(52) 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 +(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 : 18] Join type: Inner Join condition: None -(47) CometScan parquet spark_catalog.default.store_sales +(55) CometScan 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) ColumnarToRow [codegen id : 12] +(58) CometColumnarToRow [codegen id : 12] Input [1]: [ss_list_price#63] -(51) HashAggregate [codegen id : 12] +(59) 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 +(60) CometColumnarExchange +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(61) CometColumnarToRow [codegen id : 13] Input [4]: [ss_list_price#63, sum#68, count#69, count#70] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] -(53) HashAggregate [codegen id : 13] +(62) 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] +(63) 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 +(64) 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] +(65) CometColumnarToRow [codegen id : 14] +Input [4]: [sum#68, count#69, count#70, count#72] + +(66) 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 +(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 : 18] Join type: Inner Join condition: None -(59) CometScan parquet spark_catalog.default.store_sales +(69) CometScan 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) ColumnarToRow [codegen id : 15] +(72) CometColumnarToRow [codegen id : 15] Input [1]: [ss_list_price#78] -(63) HashAggregate [codegen id : 15] +(73) 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 +(74) 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] + +(75) CometColumnarToRow [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] +(76) 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] +(77) 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 +(78) CometColumnarExchange +Input [4]: [sum#83, count#84, count#85, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(79) CometColumnarToRow [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] +(80) 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 +(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 : 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 2136886fe3..da948f20fd 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,119 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (4) + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow InputAdapter - Exchange [ss_list_price] #5 - WholeStageCodegen (3) + CometColumnarExchange [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 + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow InputAdapter - Exchange [ss_list_price] #8 - WholeStageCodegen (6) + CometColumnarExchange [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 + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow InputAdapter - Exchange [ss_list_price] #11 - WholeStageCodegen (9) + CometColumnarExchange [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 + CometColumnarToRow 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] - 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] 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 6f27f77d00..6657702c42 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,50 @@ == 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 (46) ++- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -240,7 +241,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_sk#30, i_item_id#31, i_item_desc#32] Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#28, s_store_name#29, i_item_id#31, i_item_desc#32] (42) HashAggregate [codegen id : 1] @@ -250,110 +251,113 @@ 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) 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, [plan_id=1] +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] -(44) HashAggregate [codegen id : 2] +(44) CometColumnarToRow [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] + +(45) 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 +(46) 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 (51) ++- * CometColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(46) CometScan parquet spark_catalog.default.date_dim +(47) CometScan 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 +(48) 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 +(49) 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] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(50) BroadcastExchange +(51) 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 (56) ++- * CometColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(51) CometScan parquet spark_catalog.default.date_dim +(52) CometScan 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 +(53) 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 +(54) 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] +(55) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(55) BroadcastExchange +(56) 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 (61) ++- * CometColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(56) CometScan parquet spark_catalog.default.date_dim +(57) CometScan 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 +(58) 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 +(59) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(60) BroadcastExchange +(61) 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 ee1f435d50..ce745af11d 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,74 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow 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 - 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] #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] + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 + 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] 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 0b7ac89233..947a17caa3 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,24 @@ == 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 (20) ++- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -88,7 +89,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] @@ -98,18 +99,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(19) 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 +(20) 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 36bb17a161..c41606eaa7 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,24 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (1) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 f13c4c64af..6e2615311b 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,58 @@ == 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 (54) ++- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- * HashAggregate (15) + : : : +- * CometColumnarToRow (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 (38) + : : +- * Filter (37) + : : +- * HashAggregate (36) + : : +- * CometColumnarToRow (35) + : : +- CometColumnarExchange (34) + : : +- * HashAggregate (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometColumnarExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.web_returns (20) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (44) + : +- * CometColumnarToRow (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.customer (41) + +- BroadcastExchange (51) + +- * CometColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.customer_address (47) (1) CometScan parquet spark_catalog.default.web_returns @@ -116,7 +119,7 @@ Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight 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] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] (15) HashAggregate [codegen id : 1] @@ -126,22 +129,25 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 7] +(17) CometColumnarToRow [codegen id : 7] +Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] + +(18) 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] +(19) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) -(19) CometScan parquet spark_catalog.default.web_returns +(20) CometScan 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 +155,193 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(21) 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] +(22) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#21] -(22) CometBroadcastHashJoin +(23) 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 +(24) 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] +(25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#22, ca_state#23] -(25) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 2] Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] -(28) HashAggregate [codegen id : 2] +(29) 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 +(30) 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] + +(31) CometColumnarToRow [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] +(32) 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] +(33) 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 +(34) CometColumnarExchange +Input [3]: [ctr_state#26, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(35) CometColumnarToRow [codegen id : 4] Input [3]: [ctr_state#26, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(33) HashAggregate [codegen id : 4] +(36) 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] +(37) 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 +(38) 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] +(39) 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] +(40) 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) CometScan parquet spark_catalog.default.customer +(41) CometScan 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 +(42) 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] +(43) CometColumnarToRow [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 +(44) 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] +(45) 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] +(46) 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) CometScan parquet spark_catalog.default.customer_address +(47) CometScan 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 +(48) 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 +(49) CometProject Input [2]: [ca_address_sk#48, ca_state#49] Arguments: [ca_address_sk#48], [ca_address_sk#48] -(47) ColumnarToRow [codegen id : 6] +(50) CometColumnarToRow [codegen id : 6] Input [1]: [ca_address_sk#48] -(48) BroadcastExchange +(51) 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] +(52) 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] +(53) 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 +(54) 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 (59) ++- * CometColumnarToRow (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan parquet spark_catalog.default.date_dim (55) -(52) CometScan parquet spark_catalog.default.date_dim +(55) CometScan 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 +(56) 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 +(57) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(55) ColumnarToRow [codegen id : 1] +(58) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(56) BroadcastExchange +(59) 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 = 20 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 c4e59b49ac..7b1e368f27 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,69 +8,72 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (1) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + CometColumnarToRow + 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) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometProject [ca_address_sk] CometFilter [ca_address_sk,ca_state] 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 ed4a6da896..f28dd12dcd 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,100 +1,106 @@ == Physical Plan == -* ColumnarToRow (96) -+- CometSort (95) - +- CometColumnarExchange (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) +* CometColumnarToRow (102) ++- CometSort (101) + +- CometColumnarExchange (100) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * Project (83) + : +- * BroadcastHashJoin Inner BuildRight (82) + : :- * BroadcastHashJoin Inner BuildRight (67) + : : :- * Project (52) + : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : :- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * HashAggregate (17) + : : : : : +- * CometColumnarToRow (16) + : : : : : +- CometColumnarExchange (15) + : : : : : +- * HashAggregate (14) + : : : : : +- * CometColumnarToRow (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 (33) + : : : : +- * HashAggregate (32) + : : : : +- * CometColumnarToRow (31) + : : : : +- CometColumnarExchange (30) + : : : : +- * HashAggregate (29) + : : : : +- * CometColumnarToRow (28) + : : : : +- CometProject (27) + : : : : +- CometBroadcastHashJoin (26) + : : : : :- CometProject (24) + : : : : : +- CometBroadcastHashJoin (23) + : : : : : :- CometFilter (19) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (18) + : : : : : +- CometBroadcastExchange (22) + : : : : : +- CometFilter (21) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (20) + : : : : +- ReusedExchange (25) + : : : +- BroadcastExchange (50) + : : : +- * HashAggregate (49) + : : : +- * CometColumnarToRow (48) + : : : +- CometColumnarExchange (47) + : : : +- * HashAggregate (46) + : : : +- * CometColumnarToRow (45) + : : : +- CometProject (44) + : : : +- CometBroadcastHashJoin (43) + : : : :- CometProject (41) + : : : : +- CometBroadcastHashJoin (40) + : : : : :- CometFilter (36) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (35) + : : : : +- CometBroadcastExchange (39) + : : : : +- CometFilter (38) + : : : : +- CometScan parquet spark_catalog.default.date_dim (37) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (66) + : : +- * HashAggregate (65) + : : +- * CometColumnarToRow (64) + : : +- CometColumnarExchange (63) + : : +- * HashAggregate (62) + : : +- * CometColumnarToRow (61) + : : +- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometProject (57) + : : : +- CometBroadcastHashJoin (56) + : : : :- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.web_sales (53) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (58) + : +- BroadcastExchange (81) + : +- * HashAggregate (80) + : +- * CometColumnarToRow (79) + : +- CometColumnarExchange (78) + : +- * HashAggregate (77) + : +- * CometColumnarToRow (76) + : +- CometProject (75) + : +- CometBroadcastHashJoin (74) + : :- CometProject (72) + : : +- CometBroadcastHashJoin (71) + : : :- CometFilter (69) + : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : +- ReusedExchange (70) + : +- ReusedExchange (73) + +- BroadcastExchange (97) + +- * HashAggregate (96) + +- * CometColumnarToRow (95) + +- CometColumnarExchange (94) + +- * HashAggregate (93) + +- * CometColumnarToRow (92) + +- CometProject (91) + +- CometBroadcastHashJoin (90) + :- CometProject (88) + : +- CometBroadcastHashJoin (87) + : :- CometFilter (85) + : : +- CometScan parquet spark_catalog.default.web_sales (84) + : +- ReusedExchange (86) + +- ReusedExchange (89) (1) CometScan parquet spark_catalog.default.store_sales @@ -157,7 +163,7 @@ Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] (14) HashAggregate [codegen id : 1] @@ -167,18 +173,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(16) HashAggregate [codegen id : 12] +(16) CometColumnarToRow [codegen id : 12] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] + +(17) 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) CometScan parquet spark_catalog.default.store_sales +(18) CometScan 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 [] @@ -186,78 +195,81 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter +(19) CometFilter Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] Condition : isnotnull(ss_addr_sk#14) -(19) CometScan parquet spark_catalog.default.date_dim +(20) CometScan 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 +(21) 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 +(22) 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 +(23) 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 +(24) 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] +(25) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#21, ca_county#22] -(25) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 2] Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -(28) HashAggregate [codegen id : 2] +(29) 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 +(30) 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] +(31) CometColumnarToRow [codegen id : 3] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] + +(32) 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 +(33) 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] +(34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#22] Join type: Inner Join condition: None -(33) CometScan parquet spark_catalog.default.store_sales +(35) CometScan 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 [] @@ -265,82 +277,85 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter +(36) CometFilter Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Condition : isnotnull(ss_addr_sk#26) -(35) CometScan parquet spark_catalog.default.date_dim +(37) CometScan 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 +(38) 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 +(39) 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 +(40) 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 +(41) 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] +(42) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#33, ca_county#34] -(41) CometBroadcastHashJoin +(43) 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 +(44) 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] +(45) CometColumnarToRow [codegen id : 4] Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -(44) HashAggregate [codegen id : 4] +(46) 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 +(47) 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, CometColumnarShuffle, [plan_id=4] + +(48) CometColumnarToRow [codegen id : 5] 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] -(46) HashAggregate [codegen id : 5] +(49) 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 +(50) 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] +(51) 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] +(52) 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) CometScan parquet spark_catalog.default.web_sales +(53) CometScan 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 [] @@ -348,66 +363,69 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(51) CometFilter +(54) 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] +(55) ReusedExchange [Reuses operator id: 5] Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] -(53) CometBroadcastHashJoin +(56) 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 +(57) 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] +(58) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#45, ca_county#46] -(56) CometBroadcastHashJoin +(59) 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 +(60) 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] +(61) CometColumnarToRow [codegen id : 6] Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] -(59) HashAggregate [codegen id : 6] +(62) 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 +(63) 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] +(64) CometColumnarToRow [codegen id : 7] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] + +(65) 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 +(66) 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] +(67) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ca_county#9] Right keys [1]: [ca_county#46] Join type: Inner Join condition: None -(64) CometScan parquet spark_catalog.default.web_sales +(68) CometScan 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 [] @@ -415,70 +433,73 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(65) CometFilter +(69) 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] +(70) ReusedExchange [Reuses operator id: 22] Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] -(67) CometBroadcastHashJoin +(71) 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 +(72) 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] +(73) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#58, ca_county#59] -(70) CometBroadcastHashJoin +(74) 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 +(75) 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] +(76) CometColumnarToRow [codegen id : 8] Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] -(73) HashAggregate [codegen id : 8] +(77) 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 +(78) 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, CometColumnarShuffle, [plan_id=8] + +(79) CometColumnarToRow [codegen id : 9] 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] -(75) HashAggregate [codegen id : 9] +(80) 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 +(81) 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] +(82) 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] +(83) 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) CometScan parquet spark_catalog.default.web_sales +(84) CometScan 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 [] @@ -486,161 +507,164 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) CometFilter +(85) 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] +(86) ReusedExchange [Reuses operator id: 39] Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] -(82) CometBroadcastHashJoin +(87) 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 +(88) 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] +(89) ReusedExchange [Reuses operator id: 10] Output [2]: [ca_address_sk#70, ca_county#71] -(85) CometBroadcastHashJoin +(90) 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 +(91) 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] +(92) CometColumnarToRow [codegen id : 10] Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] -(88) HashAggregate [codegen id : 10] +(93) 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 +(94) 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] +(95) CometColumnarToRow [codegen id : 11] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] + +(96) 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 +(97) 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] +(98) 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] +(99) 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) CometColumnarExchange +(100) 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] -(95) CometSort +(101) 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: [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] -(96) ColumnarToRow [codegen id : 13] +(102) CometColumnarToRow [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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (100) -+- * ColumnarToRow (99) - +- CometFilter (98) - +- CometScan parquet spark_catalog.default.date_dim (97) +BroadcastExchange (106) ++- * CometColumnarToRow (105) + +- CometFilter (104) + +- CometScan parquet spark_catalog.default.date_dim (103) -(97) CometScan parquet spark_catalog.default.date_dim +(103) CometScan 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 -(98) CometFilter +(104) 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)) -(99) ColumnarToRow [codegen id : 1] +(105) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(100) BroadcastExchange +(106) 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 (104) -+- * ColumnarToRow (103) - +- CometFilter (102) - +- CometScan parquet spark_catalog.default.date_dim (101) +Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 +BroadcastExchange (110) ++- * CometColumnarToRow (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(101) CometScan parquet spark_catalog.default.date_dim +(107) CometScan 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 -(102) CometFilter +(108) 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)) -(103) ColumnarToRow [codegen id : 1] +(109) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -(104) BroadcastExchange +(110) 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 (108) -+- * ColumnarToRow (107) - +- CometFilter (106) - +- CometScan parquet spark_catalog.default.date_dim (105) +Subquery:3 Hosting operator id = 35 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (114) ++- * CometColumnarToRow (113) + +- CometFilter (112) + +- CometScan parquet spark_catalog.default.date_dim (111) -(105) CometScan parquet spark_catalog.default.date_dim +(111) CometScan 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 -(106) CometFilter +(112) 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)) -(107) ColumnarToRow [codegen id : 1] +(113) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -(108) BroadcastExchange +(114) 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 = 53 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 = 68 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 = 84 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 8a759d62f3..c99e475f07 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,5 +1,5 @@ WholeStageCodegen (13) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -13,40 +13,70 @@ WholeStageCodegen (13) 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (1) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #7 + WholeStageCodegen (2) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + BroadcastExchange #10 + WholeStageCodegen (5) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + CometColumnarToRow InputAdapter - Exchange [ca_county,d_qoy,d_year] #7 - WholeStageCodegen (2) + CometColumnarExchange [ca_county,d_qoy,d_year] #11 + WholeStageCodegen (4) HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow 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] @@ -54,53 +84,47 @@ WholeStageCodegen (13) 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 + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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 + 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 #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #15 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange [ca_county,d_qoy,d_year] #15 - WholeStageCodegen (6) + CometColumnarExchange [ca_county,d_qoy,d_year] #17 + WholeStageCodegen (8) HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow 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] @@ -108,44 +132,26 @@ WholeStageCodegen (13) 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 + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 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) - 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_county,d_qoy,d_year] #19 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 8fe97f412a..0ef65f845d 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,37 @@ == 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 (33) ++- * CometColumnarToRow (32) + +- CometColumnarExchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * CometColumnarToRow (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 (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- * CometColumnarToRow (21) + : +- CometColumnarExchange (20) + : +- * HashAggregate (19) + : +- * CometColumnarToRow (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 (27) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -72,7 +74,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] +(9) CometColumnarToRow [codegen id : 4] Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (10) CometScan parquet spark_catalog.default.catalog_sales @@ -115,7 +117,7 @@ Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] (19) HashAggregate [codegen id : 1] @@ -125,60 +127,66 @@ 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) CometColumnarExchange Input [3]: [cs_item_sk#7, sum#15, count#16] -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] +(21) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_item_sk#7, sum#15, count#16] + +(22) 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] +(23) 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 +(24) 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] +(25) 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] +(26) 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] +(27) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#19] -(27) BroadcastHashJoin [codegen id : 4] +(28) 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] +(29) 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] +(30) 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 +(31) CometColumnarExchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(32) CometColumnarToRow [codegen id : 5] Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 5] +(33) HashAggregate [codegen id : 5] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] @@ -188,32 +196,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 (38) ++- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(32) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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 +(35) 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 +(36) CometProject Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(36) BroadcastExchange +(38) 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 b6ba1fbab1..05d7a0ae5f 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,52 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #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))] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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) - 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #5 + WholeStageCodegen (1) + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + CometColumnarToRow + 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 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 2558133dee..f580dc5dac 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,71 @@ == 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 (67) ++- * HashAggregate (66) + +- * CometColumnarToRow (65) + +- CometColumnarExchange (64) + +- * HashAggregate (63) + +- Union (62) + :- * HashAggregate (29) + : +- * CometColumnarToRow (28) + : +- CometColumnarExchange (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (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 (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (34) + : : : +- CometBroadcastHashJoin (33) + : : : :- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (61) + +- * CometColumnarToRow (60) + +- CometColumnarExchange (59) + +- * HashAggregate (58) + +- * CometColumnarToRow (57) + +- CometProject (56) + +- CometBroadcastHashJoin (55) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (48) + : +- ReusedExchange (51) + +- ReusedExchange (54) (1) CometScan parquet spark_catalog.default.store_sales @@ -180,7 +184,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] (26) HashAggregate [codegen id : 1] @@ -190,18 +194,21 @@ 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) CometColumnarExchange Input [2]: [i_manufact_id#12, sum#16] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [2]: [i_manufact_id#12, sum#16] + +(29) 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) CometScan parquet spark_catalog.default.catalog_sales +(30) CometScan 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 +216,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(31) 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] +(32) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(33) 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 +(34) 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] +(35) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_manufact_id#27] -(38) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_manufact_id#27] -(41) HashAggregate [codegen id : 3] +(42) 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 +(43) CometColumnarExchange +Input [2]: [i_manufact_id#27, sum#29] +Arguments: hashpartitioning(i_manufact_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(44) CometColumnarToRow [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] +(45) 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) CometScan parquet spark_catalog.default.web_sales +(46) CometScan 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 +288,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(47) 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] +(48) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(49) 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 +(50) 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] +(51) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(52) 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 +(53) 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] +(54) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_manufact_id#40] -(53) CometBroadcastHashJoin +(55) 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 +(56) 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] +(57) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_manufact_id#40] -(56) HashAggregate [codegen id : 5] +(58) 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 +(59) CometColumnarExchange Input [2]: [i_manufact_id#40, sum#42] -Arguments: hashpartitioning(i_manufact_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_manufact_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(60) CometColumnarToRow [codegen id : 6] +Input [2]: [i_manufact_id#40, sum#42] + +(61) 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 +(62) Union -(60) HashAggregate [codegen id : 7] +(63) 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 +(64) CometColumnarExchange +Input [3]: [i_manufact_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(65) CometColumnarToRow [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] +(66) 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 +(67) 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 (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(64) CometScan parquet spark_catalog.default.date_dim +(68) CometScan 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 +(69) 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 +(70) 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) 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 = 30 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 = 46 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 bf0ef7ae47..2cf328b416 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,93 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #2 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #8 + WholeStageCodegen (3) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id] #9 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 50e3800526..452f25394a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 779d4cbddb..014786e7fd 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 c1e19555c9..74ff12aef1 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 9deb513423..8d4a8dcb97 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 2478e2e0e6..0d78366547 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,35 @@ == 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 (31) ++- * Project (30) + +- Window (29) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometColumnarExchange (26) + +- * HashAggregate (25) + +- * CometColumnarToRow (24) + +- CometColumnarExchange (23) + +- * HashAggregate (22) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -126,7 +128,7 @@ Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10], [ Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10] Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#11, i_class#10, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#11, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#14, i_class#15, spark_grouping_id#16] (22) HashAggregate [codegen id : 1] @@ -136,66 +138,72 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_category#14, i_class#15, spark_grouping_id#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) HashAggregate [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] +Input [5]: [i_category#14, i_class#15, spark_grouping_id#16, sum#19, sum#20] + +(25) 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 +(26) 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] + +(27) 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] +(28) CometColumnarToRow [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 +(29) 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] +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(33) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(34) BroadcastExchange +(36) 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 58d617430a..a95d445de1 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,44 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 844e7a729f..3f78c57bb2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 85e86ca8ae..13fcba8592 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 c0e7300dff..8139d06c08 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange 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 8272260f5c..5ce896e6e6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] 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 6e343640d6..3c1f10c183 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,53 +1,55 @@ == Physical Plan == -* ColumnarToRow (49) -+- CometSort (48) - +- CometColumnarExchange (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) +* CometColumnarToRow (51) ++- CometSort (50) + +- CometColumnarExchange (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (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 (47) + +- * Project (46) + +- * Filter (45) + +- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (42) + +- * HashAggregate (41) + +- * CometColumnarToRow (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.inventory (26) + : : +- ReusedExchange (28) + : +- ReusedExchange (31) + +- CometBroadcastExchange (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (1) CometScan parquet spark_catalog.default.inventory @@ -138,7 +140,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] (20) HashAggregate [codegen id : 1] @@ -148,26 +150,29 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 4] +(22) CometColumnarToRow [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] + +(23) 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] +(24) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#25)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#24 / knownfloatingpointnormalized(normalizenanandzero(mean#25))))) > 1.0) END -(24) Project [codegen id : 4] +(25) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#25)) = 0.0) THEN null ELSE (stdev#24 / knownfloatingpointnormalized(normalizenanandzero(mean#25))) END AS cov#26] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] -(25) CometScan parquet spark_catalog.default.inventory +(26) CometScan 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 [] @@ -175,171 +180,174 @@ PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) CometFilter +(27) 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] +(28) ReusedExchange [Reuses operator id: 5] Output [1]: [i_item_sk#32] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(31) CometBroadcastHashJoin +(32) 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 +(33) 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) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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 +(35) 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 +(36) 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 +(37) CometBroadcastExchange Input [2]: [d_date_sk#35, d_moy#37] Arguments: [d_date_sk#35, d_moy#37] -(37) CometBroadcastHashJoin +(38) 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 +(39) 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] +(40) CometColumnarToRow [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] +(41) 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 +(42) 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, CometColumnarShuffle, [plan_id=2] + +(43) CometColumnarToRow [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] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(42) HashAggregate [codegen id : 3] +(44) 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] +(45) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#49)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#48 / knownfloatingpointnormalized(normalizenanandzero(mean#49))))) > 1.0) END -(44) Project [codegen id : 3] +(46) Project [codegen id : 3] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#49)) = 0.0) THEN null ELSE (stdev#48 / knownfloatingpointnormalized(normalizenanandzero(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 +(47) 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] +(48) 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) CometColumnarExchange +(49) 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] -(48) CometSort +(50) 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: [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] -(49) ColumnarToRow [codegen id : 5] +(51) CometColumnarToRow [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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (56) ++- * CometColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(50) CometScan parquet spark_catalog.default.date_dim +(52) CometScan 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) ColumnarToRow [codegen id : 1] +(55) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(54) BroadcastExchange +(56) 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 (59) -+- * ColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan parquet spark_catalog.default.date_dim (55) +Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (61) ++- * CometColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(55) CometScan parquet spark_catalog.default.date_dim +(57) CometScan 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 -(56) CometFilter +(58) 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)) -(57) CometProject +(59) 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] -(58) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] -(59) BroadcastExchange +(61) 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 d681e8bf61..adf67bf27e 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -8,69 +8,71 @@ WholeStageCodegen (5) 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 5572232cdf..794f8afa77 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,53 +1,55 @@ == Physical Plan == -* ColumnarToRow (49) -+- CometSort (48) - +- CometColumnarExchange (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) +* CometColumnarToRow (51) ++- CometSort (50) + +- CometColumnarExchange (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (25) + : +- * Filter (24) + : +- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (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 (47) + +- * Project (46) + +- * Filter (45) + +- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (42) + +- * HashAggregate (41) + +- * CometColumnarToRow (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.inventory (26) + : : +- ReusedExchange (28) + : +- ReusedExchange (31) + +- CometBroadcastExchange (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) (1) CometScan parquet spark_catalog.default.inventory @@ -138,7 +140,7 @@ Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] (20) HashAggregate [codegen id : 1] @@ -148,26 +150,29 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 4] +(22) CometColumnarToRow [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] + +(23) 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] +(24) Filter [codegen id : 4] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#25)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#24 / knownfloatingpointnormalized(normalizenanandzero(mean#25))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#25)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#24 / knownfloatingpointnormalized(normalizenanandzero(mean#25))))) > 1.5) END) -(24) Project [codegen id : 4] +(25) Project [codegen id : 4] Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#25, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#25)) = 0.0) THEN null ELSE (stdev#24 / knownfloatingpointnormalized(normalizenanandzero(mean#25))) END AS cov#26] Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#24, mean#25] -(25) CometScan parquet spark_catalog.default.inventory +(26) CometScan 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 [] @@ -175,171 +180,174 @@ PartitionFilters: [isnotnull(inv_date_sk#30), dynamicpruningexpression(inv_date_ PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct -(26) CometFilter +(27) 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] +(28) ReusedExchange [Reuses operator id: 5] Output [1]: [i_item_sk#32] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) ReusedExchange [Reuses operator id: 10] Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] -(31) CometBroadcastHashJoin +(32) 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 +(33) 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) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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 +(35) 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 +(36) 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 +(37) CometBroadcastExchange Input [2]: [d_date_sk#35, d_moy#37] Arguments: [d_date_sk#35, d_moy#37] -(37) CometBroadcastHashJoin +(38) 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 +(39) 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] +(40) CometColumnarToRow [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] +(41) 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 +(42) 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, CometColumnarShuffle, [plan_id=2] + +(43) CometColumnarToRow [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] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(42) HashAggregate [codegen id : 3] +(44) 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] +(45) Filter [codegen id : 3] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#48, mean#49] Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#49)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#48 / knownfloatingpointnormalized(normalizenanandzero(mean#49))))) > 1.0) END -(44) Project [codegen id : 3] +(46) Project [codegen id : 3] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#49, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#49)) = 0.0) THEN null ELSE (stdev#48 / knownfloatingpointnormalized(normalizenanandzero(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 +(47) 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] +(48) 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) CometColumnarExchange +(49) 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] -(48) CometSort +(50) 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: [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] -(49) ColumnarToRow [codegen id : 5] +(51) CometColumnarToRow [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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (54) -+- * ColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan parquet spark_catalog.default.date_dim (50) +BroadcastExchange (56) ++- * CometColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) -(50) CometScan parquet spark_catalog.default.date_dim +(52) CometScan 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) ColumnarToRow [codegen id : 1] +(55) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] -(54) BroadcastExchange +(56) 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 (59) -+- * ColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan parquet spark_catalog.default.date_dim (55) +Subquery:2 Hosting operator id = 26 Hosting Expression = inv_date_sk#30 IN dynamicpruning#31 +BroadcastExchange (61) ++- * CometColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) -(55) CometScan parquet spark_catalog.default.date_dim +(57) CometScan 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 -(56) CometFilter +(58) 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)) -(57) CometProject +(59) 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] -(58) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#35, d_moy#37] -(59) BroadcastExchange +(61) 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 d681e8bf61..adf67bf27e 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -8,69 +8,71 @@ WholeStageCodegen (5) 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 bac54ba443..dcf2098b87 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,116 @@ == 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 (112) ++- * Project (111) + +- * BroadcastHashJoin Inner BuildRight (110) + :- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Project (74) + : : +- * BroadcastHashJoin Inner BuildRight (73) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Filter (18) + : : : : : +- * HashAggregate (17) + : : : : : +- * CometColumnarToRow (16) + : : : : : +- CometColumnarExchange (15) + : : : : : +- * HashAggregate (14) + : : : : : +- * CometColumnarToRow (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 (36) + : : : : +- * HashAggregate (35) + : : : : +- * CometColumnarToRow (34) + : : : : +- CometColumnarExchange (33) + : : : : +- * HashAggregate (32) + : : : : +- * CometColumnarToRow (31) + : : : : +- CometProject (30) + : : : : +- CometBroadcastHashJoin (29) + : : : : :- CometProject (25) + : : : : : +- CometBroadcastHashJoin (24) + : : : : : :- CometFilter (20) + : : : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : : : +- CometBroadcastExchange (23) + : : : : : +- CometFilter (22) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (21) + : : : : +- CometBroadcastExchange (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.date_dim (26) + : : : +- BroadcastExchange (54) + : : : +- * Filter (53) + : : : +- * HashAggregate (52) + : : : +- * CometColumnarToRow (51) + : : : +- CometColumnarExchange (50) + : : : +- * HashAggregate (49) + : : : +- * CometColumnarToRow (48) + : : : +- CometProject (47) + : : : +- CometBroadcastHashJoin (46) + : : : :- CometProject (44) + : : : : +- CometBroadcastHashJoin (43) + : : : : :- CometFilter (39) + : : : : : +- CometScan parquet spark_catalog.default.customer (38) + : : : : +- CometBroadcastExchange (42) + : : : : +- CometFilter (41) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : +- ReusedExchange (45) + : : +- BroadcastExchange (72) + : : +- * HashAggregate (71) + : : +- * CometColumnarToRow (70) + : : +- CometColumnarExchange (69) + : : +- * HashAggregate (68) + : : +- * CometColumnarToRow (67) + : : +- CometProject (66) + : : +- CometBroadcastHashJoin (65) + : : :- CometProject (63) + : : : +- CometBroadcastHashJoin (62) + : : : :- CometFilter (58) + : : : : +- CometScan parquet spark_catalog.default.customer (57) + : : : +- CometBroadcastExchange (61) + : : : +- CometFilter (60) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (59) + : : +- ReusedExchange (64) + : +- BroadcastExchange (91) + : +- * Filter (90) + : +- * HashAggregate (89) + : +- * CometColumnarToRow (88) + : +- CometColumnarExchange (87) + : +- * HashAggregate (86) + : +- * CometColumnarToRow (85) + : +- CometProject (84) + : +- CometBroadcastHashJoin (83) + : :- CometProject (81) + : : +- CometBroadcastHashJoin (80) + : : :- CometFilter (76) + : : : +- CometScan parquet spark_catalog.default.customer (75) + : : +- CometBroadcastExchange (79) + : : +- CometFilter (78) + : : +- CometScan parquet spark_catalog.default.web_sales (77) + : +- ReusedExchange (82) + +- BroadcastExchange (109) + +- * HashAggregate (108) + +- * CometColumnarToRow (107) + +- CometColumnarExchange (106) + +- * HashAggregate (105) + +- * CometColumnarToRow (104) + +- CometProject (103) + +- CometBroadcastHashJoin (102) + :- CometProject (100) + : +- CometBroadcastHashJoin (99) + : :- CometFilter (95) + : : +- CometScan parquet spark_catalog.default.customer (94) + : +- CometBroadcastExchange (98) + : +- CometFilter (97) + : +- CometScan parquet spark_catalog.default.web_sales (96) + +- ReusedExchange (101) (1) CometScan parquet spark_catalog.default.customer @@ -167,7 +173,7 @@ Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#16, d_year#17] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] (14) HashAggregate [codegen id : 1] @@ -177,33 +183,36 @@ 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) 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, [plan_id=1] +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] -(16) HashAggregate [codegen id : 12] +(16) CometColumnarToRow [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] + +(17) 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] +(18) Filter [codegen id : 12] Input [2]: [customer_id#23, year_total#24] Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) -(18) CometScan parquet spark_catalog.default.customer +(19) CometScan 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 +(20) 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) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +220,93 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(22) 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 +(23) 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 +(24) 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 +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometBroadcastExchange Input [2]: [d_date_sk#40, d_year#41] Arguments: [d_date_sk#40, d_year#41] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) CometColumnarToRow [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] +(32) 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 +(33) 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, [plan_id=2] +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] -(33) HashAggregate [codegen id : 3] +(34) CometColumnarToRow [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] + +(35) 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 +(36) 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] +(37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [customer_id#23] Right keys [1]: [customer_id#46] Join type: Inner Join condition: None -(36) CometScan parquet spark_catalog.default.customer +(38) CometScan 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 +(39) 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) CometScan parquet spark_catalog.default.catalog_sales +(40) CometScan 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 +314,89 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(41) 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 +(42) 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 +(43) 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#69, d_year#70] -(44) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) CometColumnarToRow [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] +(49) 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 +(50) 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] + +(51) CometColumnarToRow [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] +(52) 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] +(53) 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 +(54) 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] +(55) 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] +(56) 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) CometScan parquet spark_catalog.default.customer +(57) CometScan 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 +(58) 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) CometScan parquet spark_catalog.default.catalog_sales +(59) CometScan 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 +404,85 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(60) 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 +(61) 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 +(62) 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 +(63) 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] +(64) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#93, d_year#94] -(62) CometBroadcastHashJoin +(65) 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 +(66) 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] +(67) CometColumnarToRow [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] +(68) 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 +(69) 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, CometColumnarShuffle, [plan_id=6] + +(70) CometColumnarToRow [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] -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] -(67) HashAggregate [codegen id : 7] +(71) 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 +(72) 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] +(73) 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] +(74) 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) CometScan parquet spark_catalog.default.customer +(75) CometScan 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 +(76) 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) CometScan parquet spark_catalog.default.web_sales +(77) CometScan 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 +490,89 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(74) CometFilter +(78) 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 +(79) 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 +(80) 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 +(81) 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] +(82) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#116, d_year#117] -(79) CometBroadcastHashJoin +(83) 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 +(84) 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] +(85) CometColumnarToRow [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] +(86) 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 +(87) 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] +(88) CometColumnarToRow [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] + +(89) 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] +(90) 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 +(91) 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] +(92) 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] +(93) 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) CometScan parquet spark_catalog.default.customer +(94) CometScan 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 +(95) 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) CometScan parquet spark_catalog.default.web_sales +(96) CometScan 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 +580,135 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(92) CometFilter +(97) 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 +(98) 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 +(99) 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 +(100) 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] +(101) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#140, d_year#141] -(97) CometBroadcastHashJoin +(102) 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 +(103) 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] +(104) CometColumnarToRow [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] +(105) 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 +(106) 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] + +(107) CometColumnarToRow [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] +(108) 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 +(109) 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] +(110) 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] +(111) 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 +(112) 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 (116) ++- * CometColumnarToRow (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) -(107) CometScan parquet spark_catalog.default.date_dim +(113) CometScan 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 +(114) 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] +(115) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] -(110) BroadcastExchange +(116) 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 = 21 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (120) ++- * CometColumnarToRow (119) + +- CometFilter (118) + +- CometScan parquet spark_catalog.default.date_dim (117) -(111) CometScan parquet spark_catalog.default.date_dim +(117) CometScan 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 +(118) 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] +(119) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#40, d_year#41] -(114) BroadcastExchange +(120) 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 = 40 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 = 59 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 = 77 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 = 96 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 a412a54d9f..826222bd0c 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,143 +11,149 @@ 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 908b4909e5..f93089ffb2 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,36 +1,37 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- Exchange (30) - +- * HashAggregate (29) - +- * ColumnarToRow (28) - +- CometProject (27) - +- CometBroadcastHashJoin (26) - :- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (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) +TakeOrderedAndProject (33) ++- * HashAggregate (32) + +- * CometColumnarToRow (31) + +- CometColumnarExchange (30) + +- * HashAggregate (29) + +- * CometColumnarToRow (28) + +- CometProject (27) + +- CometBroadcastHashJoin (26) + :- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- CometProject (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (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) CometScan parquet spark_catalog.default.catalog_sales @@ -161,7 +162,7 @@ Arguments: [cs_sold_date_sk#5], [d_date_sk#16], Inner, BuildRight 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] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [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] @@ -171,45 +172,48 @@ Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_pr 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] -(30) Exchange +(30) 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, [plan_id=3] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(31) HashAggregate [codegen id : 2] +(31) CometColumnarToRow [codegen id : 2] +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(32) 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] -(32) TakeOrderedAndProject +(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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * ColumnarToRow (35) - +- CometFilter (34) - +- CometScan parquet spark_catalog.default.date_dim (33) +BroadcastExchange (37) ++- * CometColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(33) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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 -(34) CometFilter +(35) 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)) -(35) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] -(36) BroadcastExchange +(37) BroadcastExchange Input [2]: [d_date_sk#16, d_date#17] 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 25a80286d5..9334a2d9ac 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,43 +1,44 @@ TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] 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 (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] - CometExchange [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] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [w_state,i_item_id] #1 + WholeStageCodegen (1) + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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-spark4_0/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt index 48a1c60d16..bb3c74daeb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] 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] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [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 007103499b..de12ce9581 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] 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 724b1e8847..9a4446d858 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,24 @@ == 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 (20) ++- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -88,7 +89,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] (16) HashAggregate [codegen id : 1] @@ -98,18 +99,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] + +(19) 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 +(20) 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 d6e3733ce2..d8c6e1f74b 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,24 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_category_id,i_category] #1 + WholeStageCodegen (1) + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 016788b64c..c2d1759edd 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,24 @@ == 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 (20) ++- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -88,7 +89,7 @@ Arguments: [ss_store_sk#4], [s_store_sk#7], Inner, BuildRight Input [6]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5, s_store_sk#7, s_store_id#8, s_store_name#9] Arguments: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9], [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] (16) HashAggregate [codegen id : 1] @@ -98,18 +99,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [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] + +(19) 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 +(20) 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 f21d846c0f..3cbf72c1c6 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,24 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 c9e28476b6..362247da01 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,51 @@ == 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) +* CometColumnarToRow (47) ++- CometTakeOrderedAndProject (46) + +- CometProject (45) + +- CometBroadcastHashJoin (44) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (37) + : : +- CometSortMergeJoin (36) + : : :- CometSort (20) + : : : +- CometColumnarExchange (19) + : : : +- * Project (18) + : : : +- * Filter (17) + : : : +- Window (16) + : : : +- WindowGroupLimit (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometColumnarExchange (12) + : : : +- WindowGroupLimit (11) + : : : +- * Sort (10) + : : : +- * Filter (9) + : : : +- * HashAggregate (8) + : : : +- * CometColumnarToRow (7) + : : : +- CometColumnarExchange (6) + : : : +- * HashAggregate (5) + : : : +- * CometColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- CometSort (35) + : : +- CometColumnarExchange (34) + : : +- * Project (33) + : : +- * Filter (32) + : : +- Window (31) + : : +- WindowGroupLimit (30) + : : +- * CometColumnarToRow (29) + : : +- CometSort (28) + : : +- CometColumnarExchange (27) + : : +- WindowGroupLimit (26) + : : +- * Sort (25) + : : +- * Filter (24) + : : +- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- ReusedExchange (21) + : +- CometBroadcastExchange (40) + : +- CometFilter (39) + : +- CometScan parquet spark_catalog.default.item (38) + +- ReusedExchange (43) (1) CometScan parquet spark_catalog.default.store_sales @@ -59,7 +63,7 @@ Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [2]: [ss_item_sk#1, ss_net_profit#3] (5) HashAggregate [codegen id : 1] @@ -69,218 +73,231 @@ 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) CometColumnarExchange Input [3]: [ss_item_sk#1, sum#7, count#8] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(7) HashAggregate [codegen id : 2] +(7) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_item_sk#1, sum#7, count#8] + +(8) 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] +(9) 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] +(10) Sort [codegen id : 2] Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 -(10) WindowGroupLimit +(11) WindowGroupLimit Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial -(11) Exchange +(12) CometColumnarExchange Input [2]: [item_sk#10, rank_col#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(12) Sort [codegen id : 3] +(13) CometSort +Input [2]: [item_sk#10, rank_col#11] +Arguments: [item_sk#10, rank_col#11], [rank_col#11 ASC NULLS FIRST] + +(14) CometColumnarToRow [codegen id : 3] Input [2]: [item_sk#10, rank_col#11] -Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 -(13) WindowGroupLimit +(15) WindowGroupLimit Input [2]: [item_sk#10, rank_col#11] Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final -(14) Window +(16) 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] +(17) 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] +(18) Project [codegen id : 4] Output [2]: [item_sk#10, rnk#14] Input [3]: [item_sk#10, rank_col#11, rnk#14] -(17) Exchange +(19) 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] +(20) 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] +(21) ReusedExchange [Reuses operator id: 6] Output [3]: [ss_item_sk#15, sum#16, count#17] -(20) HashAggregate [codegen id : 7] +(22) CometColumnarToRow [codegen id : 6] +Input [3]: [ss_item_sk#15, sum#16, count#17] + +(23) 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] +(24) 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] +(25) Sort [codegen id : 6] Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], false, 0 -(23) WindowGroupLimit +(26) WindowGroupLimit Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Partial -(24) Exchange +(27) CometColumnarExchange Input [2]: [item_sk#20, rank_col#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(25) Sort [codegen id : 8] +(28) CometSort Input [2]: [item_sk#20, rank_col#21] -Arguments: [rank_col#21 DESC NULLS LAST], false, 0 +Arguments: [item_sk#20, rank_col#21], [rank_col#21 DESC NULLS LAST] -(26) WindowGroupLimit +(29) CometColumnarToRow [codegen id : 7] +Input [2]: [item_sk#20, rank_col#21] + +(30) WindowGroupLimit Input [2]: [item_sk#20, rank_col#21] Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Final -(27) Window +(31) 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] +(32) 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] +(33) Project [codegen id : 8] Output [2]: [item_sk#20, rnk#22] Input [3]: [item_sk#20, rank_col#21, rnk#22] -(30) Exchange +(34) CometColumnarExchange Input [2]: [item_sk#20, rnk#22] -Arguments: hashpartitioning(rnk#22, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(rnk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(31) Sort [codegen id : 10] +(35) CometSort Input [2]: [item_sk#20, rnk#22] -Arguments: [rnk#22 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#20, rnk#22], [rnk#22 ASC NULLS FIRST] -(32) SortMergeJoin [codegen id : 13] -Left keys [1]: [rnk#14] -Right keys [1]: [rnk#22] -Join type: Inner -Join condition: None +(36) 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] +(37) 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) CometScan parquet spark_catalog.default.item +(38) CometScan 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 +(39) 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 +(40) 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 +(41) 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] +(42) 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] +(43) ReusedExchange [Reuses operator id: 40] 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 +(44) 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] +(45) 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] -(43) TakeOrderedAndProject +(46) 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] + +(47) CometColumnarToRow [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 = 9 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +* HashAggregate (55) ++- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * CometColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.store_sales (48) -(44) CometScan parquet spark_catalog.default.store_sales +(48) CometScan 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 +(49) 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 +(50) 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] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [ss_store_sk#30, ss_net_profit#31] -(48) HashAggregate [codegen id : 1] +(52) 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 +(53) CometColumnarExchange +Input [3]: [ss_store_sk#30, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(54) CometColumnarToRow [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] +(55) 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 = 24 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 12c8e0af9f..c0753f588b 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,26 +1,26 @@ -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 (4) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (3) - Sort [rank_col] - InputAdapter - Exchange #2 +WholeStageCodegen (9) + CometColumnarToRow + 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 + WholeStageCodegen (4) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometColumnarExchange #2 WindowGroupLimit [rank_col] WholeStageCodegen (2) Sort [rank_col] @@ -28,54 +28,50 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk] #4 + WholeStageCodegen (1) + HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] + CometColumnarToRow + 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] - 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) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (8) - Sort [rank_col] - InputAdapter - Exchange #6 + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] + CometColumnarToRow + 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 + WholeStageCodegen (8) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometColumnarExchange #6 WindowGroupLimit [rank_col] - WholeStageCodegen (7) + 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] - 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 + CometColumnarToRow + 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 962a51203b..d785f36b6a 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,41 @@ == 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 (37) ++- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) CometScan parquet spark_catalog.default.web_sales @@ -149,7 +150,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +168,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -195,50 +196,53 @@ 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) CometColumnarExchange Input [3]: [ca_zip#11, ca_city#10, sum#20] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(35) HashAggregate [codegen id : 3] +(35) CometColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(36) 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 +(37) 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 (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(37) CometScan parquet spark_catalog.default.date_dim +(38) CometScan 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 +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(42) 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 f60fdb18a1..f43d2a0146 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,52 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 a09bd7640d..32b250c483 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,47 @@ == 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 (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * HashAggregate (30) + : : +- * CometColumnarToRow (29) + : : +- CometColumnarExchange (28) + : : +- * HashAggregate (27) + : : +- * CometColumnarToRow (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 (34) + : +- * CometColumnarToRow (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.customer (31) + +- BroadcastExchange (40) + +- * CometColumnarToRow (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.customer_address (37) (1) CometScan parquet spark_catalog.default.store_sales @@ -163,7 +164,7 @@ Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] (27) HashAggregate [codegen id : 1] @@ -173,106 +174,109 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(29) HashAggregate [codegen id : 4] +(29) CometColumnarToRow [codegen id : 4] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#22, sum#23] + +(30) 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) CometScan parquet spark_catalog.default.customer +(31) CometScan 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 +(32) 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] +(33) CometColumnarToRow [codegen id : 2] Input [4]: [c_customer_sk#29, c_current_addr_sk#30, c_first_name#31, c_last_name#32] -(33) BroadcastExchange +(34) 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] +(35) 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] +(36) 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) CometScan parquet spark_catalog.default.customer_address +(37) CometScan 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 +(38) 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] +(39) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#33, ca_city#34] -(39) BroadcastExchange +(40) 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] +(41) 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] +(42) 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 +(43) 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 (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan 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 +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(48) 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 9ce3a98eb5..bc02f1ceca 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,56 +5,57 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index 170424a150..fbd51f9007 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,54 @@ == 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 (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (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 (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan parquet spark_catalog.default.item @@ -130,7 +135,7 @@ Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] (19) HashAggregate [codegen id : 1] @@ -140,142 +145,157 @@ 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) 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, [plan_id=1] +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] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [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] + +(22) 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 +(23) 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, CometColumnarShuffle, [plan_id=2] + +(24) 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: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] +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] -(23) Sort [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(24) Window +(26) 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] +(27) 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 +(28) 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] +(29) 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] +(30) 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] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(30) HashAggregate [codegen id : 6] +(32) CometColumnarToRow [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] + +(33) 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 +(34) 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, CometColumnarShuffle, [plan_id=3] + +(35) 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: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=3] +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] -(32) Sort [codegen id : 7] +(36) CometColumnarToRow [codegen id : 7] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 -(33) Window +(37) 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] +(38) 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 +(39) 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] +(40) 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] +(41) 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] +(42) ReusedExchange [Reuses operator id: 34] 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] +(43) 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, 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] + +(44) CometColumnarToRow [codegen id : 11] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 -(40) Window +(45) 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] +(46) 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 +(47) 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] +(48) 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] +(49) 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 +(50) 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 (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(46) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(54) 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 1db8e7528d..2702c0f44a 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,43 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 +57,15 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + 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 +73,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] + CometColumnarToRow 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 15b16c1e8f..6d51aed6b8 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,33 @@ == 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 (29) ++- * CometColumnarToRow (28) + +- CometColumnarExchange (27) + +- * HashAggregate (26) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -145,7 +146,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#16], Inner, BuildRight Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#16] Arguments: [ss_quantity#4], [ss_quantity#4] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [1]: [ss_quantity#4] (26) HashAggregate [codegen id : 1] @@ -155,11 +156,14 @@ Functions [1]: [partial_sum(ss_quantity#4)] Aggregate Attributes [1]: [sum#18] Results [1]: [sum#19] -(27) Exchange +(27) CometColumnarExchange Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [1]: [sum#19] + +(29) HashAggregate [codegen id : 2] Input [1]: [sum#19] Keys: [] Functions [1]: [sum(ss_quantity#4)] @@ -169,32 +173,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 (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) CometScan parquet spark_catalog.default.date_dim +(30) CometScan 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 +(31) 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 +(32) CometProject Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] -(33) BroadcastExchange +(34) 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 23e0835359..618ef897b0 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,41 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ss_quantity] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 205e71071f..aa3d6bdae2 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,88 @@ == 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) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- Union (79) + :- * Project (28) + : +- * Filter (27) + : +- Window (26) + : +- * Sort (25) + : +- Window (24) + : +- * CometColumnarToRow (23) + : +- CometSort (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (19) + : +- CometColumnarExchange (18) + : +- * HashAggregate (17) + : +- * CometColumnarToRow (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 (53) + : +- * Filter (52) + : +- Window (51) + : +- * Sort (50) + : +- Window (49) + : +- * CometColumnarToRow (48) + : +- CometSort (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometBroadcastExchange (32) + : : : +- CometProject (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.catalog_returns (33) + : +- ReusedExchange (38) + +- * Project (78) + +- * Filter (77) + +- Window (76) + +- * Sort (75) + +- Window (74) + +- * CometColumnarToRow (73) + +- CometSort (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * CometColumnarToRow (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometBroadcastExchange (57) + : : +- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.store_sales (54) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometScan parquet spark_catalog.default.store_returns (58) + +- ReusedExchange (63) (1) CometScan parquet spark_catalog.default.web_sales @@ -150,7 +157,7 @@ Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] (17) HashAggregate [codegen id : 1] @@ -160,46 +167,52 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] + +(20) 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 +(21) CometColumnarExchange +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(22) CometSort Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [item#32, return_ratio#33, currency_ratio#34], [return_ratio#33 ASC NULLS FIRST] -(21) Sort [codegen id : 3] +(23) CometColumnarToRow [codegen id : 3] Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 -(22) Window +(24) 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] +(25) 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 +(26) 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] +(27) 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] +(28) 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) CometScan parquet spark_catalog.default.catalog_sales +(29) CometScan 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 +220,110 @@ 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 +(30) 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 +(31) 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 +(32) 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) CometScan parquet spark_catalog.default.catalog_returns +(33) CometScan 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 +(34) 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 +(35) 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 +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#50] -(37) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) CometColumnarToRow [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] +(42) 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 +(43) 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] + +(44) CometColumnarToRow [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] +(45) 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 +(46) CometColumnarExchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(47) 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] +(48) CometColumnarToRow [codegen id : 8] Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(45) Window +(49) 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] +(50) 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 +(51) 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] +(52) 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] +(53) 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) CometScan parquet spark_catalog.default.store_sales +(54) CometScan 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 +331,168 @@ 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 +(55) 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 +(56) 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 +(57) 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) CometScan parquet spark_catalog.default.store_returns +(58) CometScan 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 +(59) 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 +(60) 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 +(61) 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 +(62) 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] +(63) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#85] -(60) CometBroadcastHashJoin +(64) 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 +(65) 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] +(66) CometColumnarToRow [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] +(67) 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 +(68) 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, CometColumnarShuffle, [plan_id=5] + +(69) CometColumnarToRow [codegen id : 12] Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(65) HashAggregate [codegen id : 12] +(70) 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 +(71) 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] +(72) 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 +(73) CometColumnarToRow [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(74) 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] +(75) 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 +(76) 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] +(77) 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] +(78) 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 +(79) Union -(74) HashAggregate [codegen id : 16] +(80) 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 +(81) 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] +(82) 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 +(83) 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] + +(84) CometColumnarToRow [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 (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan parquet spark_catalog.default.date_dim (85) -(78) CometScan parquet spark_catalog.default.date_dim +(85) CometScan 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 +(86) 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 +(87) 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] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(89) 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 = 29 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 = 54 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 bd3b21cdbd..69f6f217e6 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,128 @@ -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] - 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] - 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] - 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 +WholeStageCodegen (17) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 a29ef4cec3..7996ca9d88 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,78 @@ == 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 (74) ++- * HashAggregate (73) + +- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * Expand (69) + +- Union (68) + :- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (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 (43) + : +- * CometColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometUnion (30) + : : : :- CometProject (26) + : : : : +- CometFilter (25) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (27) + : : +- ReusedExchange (31) + : +- CometBroadcastExchange (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.catalog_page (34) + +- * HashAggregate (67) + +- * CometColumnarToRow (66) + +- CometColumnarExchange (65) + +- * HashAggregate (64) + +- * CometColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometProject (57) + : +- CometBroadcastHashJoin (56) + : :- CometUnion (54) + : : :- CometProject (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- CometProject (53) + : : +- CometBroadcastHashJoin (52) + : : :- CometBroadcastExchange (48) + : : : +- CometScan parquet spark_catalog.default.web_returns (47) + : : +- CometProject (51) + : : +- CometFilter (50) + : : +- CometScan parquet spark_catalog.default.web_sales (49) + : +- ReusedExchange (55) + +- CometBroadcastExchange (60) + +- CometFilter (59) + +- CometScan parquet spark_catalog.default.web_site (58) (1) CometScan parquet spark_catalog.default.store_sales @@ -159,7 +163,7 @@ Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] (20) HashAggregate [codegen id : 1] @@ -169,18 +173,21 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(23) 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) CometScan parquet spark_catalog.default.catalog_sales +(24) CometScan 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 +195,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(25) 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 +(26) 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) CometScan parquet spark_catalog.default.catalog_returns +(27) CometScan 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 +211,79 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(28) 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 +(29) 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 +(30) 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] +(31) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin +(32) 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 +(33) 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) CometScan parquet spark_catalog.default.catalog_page +(34) CometScan 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 +(35) CometFilter Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(35) CometBroadcastExchange +(36) 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 +(37) 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 +(38) 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] +(39) CometColumnarToRow [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] +(40) 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 +(41) 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, [plan_id=2] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(41) HashAggregate [codegen id : 4] +(42) CometColumnarToRow [codegen id : 4] +Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] + +(43) 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) CometScan parquet spark_catalog.default.web_sales +(44) CometScan 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 +291,184 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter +(45) 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 +(46) 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) CometScan parquet spark_catalog.default.web_returns +(47) CometScan 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 +(48) 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) CometScan parquet spark_catalog.default.web_sales +(49) CometScan 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 +(50) 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 +(51) 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 +(52) 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 +(53) 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 +(54) 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] +(55) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#110] -(54) CometBroadcastHashJoin +(56) 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 +(57) 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) CometScan parquet spark_catalog.default.web_site +(58) CometScan 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 +(59) CometFilter Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(58) CometBroadcastExchange +(60) CometBroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: [web_site_sk#111, web_site_id#112] -(59) CometBroadcastHashJoin +(61) 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 +(62) 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] +(63) CometColumnarToRow [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] +(64) 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 +(65) 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] + +(66) CometColumnarToRow [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] +(67) 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 +(68) Union -(66) Expand [codegen id : 7] +(69) 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] +(70) 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 +(71) 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] + +(72) CometColumnarToRow [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] +(73) 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 +(74) 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 (79) ++- * CometColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) -(71) CometScan parquet spark_catalog.default.date_dim +(75) CometScan 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 +(76) 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 +(77) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(79) 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 = 24 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 = 27 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 = 44 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 = 47 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 ceb3e92131..6696561054 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,103 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 c55f258e53..f565757115 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,33 @@ == 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 (29) ++- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- CometColumnarExchange (26) + +- * HashAggregate (25) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -142,7 +143,7 @@ Arguments: [sr_returned_date_sk#9], [d_date_sk#23], Inner, BuildRight Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] (25) HashAggregate [codegen id : 1] @@ -152,50 +153,53 @@ 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) 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, [plan_id=1] +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] -(27) HashAggregate [codegen id : 2] +(27) CometColumnarToRow [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] + +(28) 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 +(29) 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 (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) -(29) CometScan parquet spark_catalog.default.date_dim +(30) CometScan 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 +(31) 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 +(32) 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] +(33) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] -(33) BroadcastExchange +(34) 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 aa3f7f2cf8..c596375949 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,41 @@ 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] - 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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_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) + CometColumnarToRow + 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 78c30636d6..043d3a9f6f 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,49 @@ == 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) +TakeOrderedAndProject (45) ++- * Filter (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- CometExchange (40) + +- CometProject (39) + +- CometSortMergeJoin (38) + :- CometSort (20) + : +- CometColumnarExchange (19) + : +- * Project (18) + : +- Window (17) + : +- * CometColumnarToRow (16) + : +- CometSort (15) + : +- CometColumnarExchange (14) + : +- * HashAggregate (13) + : +- * CometColumnarToRow (12) + : +- CometColumnarExchange (11) + : +- * HashAggregate (10) + : +- * CometColumnarToRow (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 (37) + +- CometColumnarExchange (36) + +- * Project (35) + +- Window (34) + +- * CometColumnarToRow (33) + +- CometSort (32) + +- CometColumnarExchange (31) + +- * HashAggregate (30) + +- * CometColumnarToRow (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- CometProject (25) + +- CometBroadcastHashJoin (24) + :- CometFilter (22) + : +- CometScan parquet spark_catalog.default.store_sales (21) + +- ReusedExchange (23) (1) CometScan parquet spark_catalog.default.web_sales @@ -81,7 +86,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] (10) HashAggregate [codegen id : 1] @@ -91,42 +96,48 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) HashAggregate [codegen id : 2] +(12) CometColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] + +(13) 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 +(14) 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] +(15) 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 +(16) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, _w0#12, ws_item_sk#1] + +(17) 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] +(18) 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 +(19) CometColumnarExchange 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] +Arguments: hashpartitioning(item_sk#11, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) Sort [codegen id : 5] +(20) CometSort 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: [item_sk#11, d_date#6, cume_sales#13], [item_sk#11 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(19) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +145,137 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(22) 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] +(23) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#18, d_date#19] -(22) CometBroadcastHashJoin +(24) 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 +(25) 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] +(26) CometColumnarToRow [codegen id : 5] Input [3]: [ss_item_sk#14, ss_sales_price#15, d_date#19] -(25) HashAggregate [codegen id : 6] +(27) 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 +(28) 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] + +(29) CometColumnarToRow [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] +(30) 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 +(31) CometColumnarExchange +Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] +Arguments: hashpartitioning(ss_item_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(32) CometSort 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] +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] -(29) Sort [codegen id : 8] +(33) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#23, d_date#19, _w0#24, ss_item_sk#14] -Arguments: [ss_item_sk#14 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 -(30) Window +(34) 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] +(35) 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 +(36) 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] +(37) 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 +(38) 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] +(39) 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] + +(40) CometExchange +Input [4]: [item_sk#26, d_date#27, web_sales#28, store_sales#29] +Arguments: hashpartitioning(item_sk#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(36) Exchange +(41) CometSort 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: [item_sk#26, d_date#27, web_sales#28, store_sales#29], [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST] -(37) Sort [codegen id : 12] +(42) CometColumnarToRow [codegen id : 9] 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 -(38) Window +(43) Window Input [4]: [item_sk#26, d_date#27, web_sales#28, store_sales#29] Arguments: [max(web_sales#28) windowspecdefinition(item_sk#26, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#30, max(store_sales#29) windowspecdefinition(item_sk#26, d_date#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#31], [item_sk#26], [d_date#27 ASC NULLS FIRST] -(39) Filter [codegen id : 13] +(44) Filter [codegen id : 10] 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 +(45) TakeOrderedAndProject Input [6]: [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] Arguments: 100, [item_sk#26 ASC NULLS FIRST, d_date#27 ASC NULLS FIRST], [item_sk#26, d_date#27, web_sales#28, store_sales#29, web_cumulative#30, store_cumulative#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (45) -+- * ColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan parquet spark_catalog.default.date_dim (41) +BroadcastExchange (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(41) CometScan parquet spark_catalog.default.date_dim +(46) CometScan 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 +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(50) 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 = 21 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 f19312ca87..80de73f470 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,76 +1,74 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (13) + WholeStageCodegen (10) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (12) - Sort [item_sk,d_date] + WholeStageCodegen (9) + CometColumnarToRow 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] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - 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) - 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 #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] - InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (9) - Project [item_sk,d_date,cume_sales] - 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) - 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] #6 + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk,d_date] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + CometColumnarToRow + 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 #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 + WholeStageCodegen (8) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + WholeStageCodegen (6) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_item_sk,d_date] #9 + WholeStageCodegen (5) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + CometColumnarToRow + 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] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt index 9e017b9279..f075ef57e0 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,24 @@ == 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 (20) ++- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -88,7 +89,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] @@ -98,18 +99,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(19) 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 +(20) 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 d7661b7bef..6a258c02b5 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,24 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (1) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 631d21aa56..84837a8d46 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,35 @@ == 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 (31) ++- * Project (30) + +- * Filter (29) + +- Window (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometColumnarExchange (25) + +- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.item @@ -122,7 +124,7 @@ Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] (21) HashAggregate [codegen id : 1] @@ -132,70 +134,76 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarToRow [codegen id : 2] +Input [3]: [i_manufact_id#5, d_qoy#17, sum#20] + +(24) 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 +(25) 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] + +(26) 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] +(27) CometColumnarToRow [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 +(28) 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] +(29) 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] +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) 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] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] -(34) BroadcastExchange +(36) 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 b90cb42d02..16db6f8433 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,43 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (3) - Sort [i_manufact_id] + CometColumnarToRow 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 + WholeStageCodegen (2) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 a1fce65e88..6bde5a7c6d 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,62 @@ == Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- Exchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- Exchange (51) - +- * HashAggregate (50) - +- * ColumnarToRow (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (37) - : : +- CometBroadcastHashJoin (36) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (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) +TakeOrderedAndProject (58) ++- * HashAggregate (57) + +- * CometColumnarToRow (56) + +- CometColumnarExchange (55) + +- * HashAggregate (54) + +- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- * CometColumnarToRow (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometExchange (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) CometScan parquet spark_catalog.default.catalog_sales @@ -288,7 +290,7 @@ Arguments: [ss_sold_date_sk#24], [d_date_sk#31], Inner, BuildRight 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] +(49) CometColumnarToRow [codegen id : 1] Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] (50) HashAggregate [codegen id : 1] @@ -298,189 +300,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] Aggregate Attributes [1]: [sum#37] Results [2]: [c_customer_sk#20, sum#38] -(51) Exchange +(51) CometColumnarExchange Input [2]: [c_customer_sk#20, sum#38] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(52) HashAggregate [codegen id : 2] +(52) CometColumnarToRow [codegen id : 2] +Input [2]: [c_customer_sk#20, sum#38] + +(53) 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))#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 : 2] +(54) HashAggregate [codegen id : 2] Input [1]: [segment#40] Keys [1]: [segment#40] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#41] Results [2]: [segment#40, count#42] -(54) Exchange +(55) CometColumnarExchange +Input [2]: [segment#40, count#42] +Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(56) CometColumnarToRow [codegen id : 3] Input [2]: [segment#40, count#42] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(55) HashAggregate [codegen id : 3] +(57) HashAggregate [codegen id : 3] Input [2]: [segment#40, count#42] Keys [1]: [segment#40] Functions [1]: [count(1)] 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 +(58) 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 (63) ++- * CometColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.date_dim (59) -(57) CometScan parquet spark_catalog.default.date_dim +(59) CometScan 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 +(60) 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 +(61) 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] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(61) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 Subquery:3 Hosting operator id = 28 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) +BroadcastExchange (68) ++- * CometColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(62) CometScan parquet spark_catalog.default.date_dim +(64) CometScan 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 -(63) CometFilter +(65) 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 +(66) 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] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] -(66) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#31] 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#33, [id=#35] +Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#33, [id=#35] -Subquery:5 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] +Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#34, [id=#36] -Subquery:6 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#33, [id=#35] -* ColumnarToRow (73) -+- CometHashAggregate (72) - +- CometExchange (71) - +- CometHashAggregate (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan parquet spark_catalog.default.date_dim (67) +Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#33, [id=#35] +* CometColumnarToRow (75) ++- CometHashAggregate (74) + +- CometExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan parquet spark_catalog.default.date_dim (69) -(67) CometScan parquet spark_catalog.default.date_dim +(69) CometScan 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 +(70) 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 +(71) 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 +(72) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(71) CometExchange +(73) CometExchange Input [1]: [(d_month_seq + 1)#49] Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(72) CometHashAggregate +(74) CometHashAggregate Input [1]: [(d_month_seq + 1)#49] Keys [1]: [(d_month_seq + 1)#49] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [(d_month_seq + 1)#49] -Subquery:7 Hosting operator id = 62 Hosting Expression = Subquery scalar-subquery#34, [id=#36] -* ColumnarToRow (80) -+- CometHashAggregate (79) - +- CometExchange (78) - +- CometHashAggregate (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan parquet spark_catalog.default.date_dim (74) +Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#34, [id=#36] +* CometColumnarToRow (82) ++- CometHashAggregate (81) + +- CometExchange (80) + +- CometHashAggregate (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(74) CometScan parquet spark_catalog.default.date_dim +(76) CometScan 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 +(77) 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 +(78) 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 +(79) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(78) CometExchange +(80) CometExchange Input [1]: [(d_month_seq + 3)#53] Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(79) CometHashAggregate +(81) CometHashAggregate Input [1]: [(d_month_seq + 3)#53] Keys [1]: [(d_month_seq + 3)#53] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [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] 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 1dddb45efc..639bc2f168 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,105 +1,107 @@ TakeOrderedAndProject [segment,num_customers,segment_base] WholeStageCodegen (3) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - InputAdapter - Exchange [segment] #1 - WholeStageCodegen (2) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - InputAdapter - Exchange [c_customer_sk] #2 - 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] - CometExchange [c_customer_sk,c_current_addr_sk] #3 + CometColumnarToRow + InputAdapter + CometColumnarExchange [segment] #1 + WholeStageCodegen (2) + HashAggregate [segment] [count,count] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_sk] #2 + WholeStageCodegen (1) + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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] - 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)] - CometExchange [(d_month_seq + 1)] #10 + CometExchange [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) + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 + CometExchange [(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) + CometColumnarToRow + 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] - 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 + CometExchange [(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] + 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 fc018e3d16..9dbe946610 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,24 @@ == 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 (20) ++- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.date_dim @@ -88,7 +89,7 @@ Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9], [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] (16) HashAggregate [codegen id : 1] @@ -98,18 +99,21 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] +Input [3]: [i_brand#9, i_brand_id#8, sum#12] + +(19) 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 +(20) 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 2461ab830b..864f95002b 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,24 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand,i_brand_id] #1 + WholeStageCodegen (1) + HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 47fc0065ef..8a148ce5b4 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,71 @@ == 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 (67) ++- * HashAggregate (66) + +- * CometColumnarToRow (65) + +- CometColumnarExchange (64) + +- * HashAggregate (63) + +- Union (62) + :- * HashAggregate (29) + : +- * CometColumnarToRow (28) + : +- CometColumnarExchange (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (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 (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (34) + : : : +- CometBroadcastHashJoin (33) + : : : :- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (61) + +- * CometColumnarToRow (60) + +- CometColumnarExchange (59) + +- * HashAggregate (58) + +- * CometColumnarToRow (57) + +- CometProject (56) + +- CometBroadcastHashJoin (55) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (48) + : +- ReusedExchange (51) + +- ReusedExchange (54) (1) CometScan parquet spark_catalog.default.store_sales @@ -180,7 +184,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#12] (26) HashAggregate [codegen id : 1] @@ -190,18 +194,21 @@ 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) CometColumnarExchange Input [2]: [i_item_id#12, sum#16] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] + +(29) 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) CometScan parquet spark_catalog.default.catalog_sales +(30) CometScan 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 +216,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(31) 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] +(32) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(33) 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 +(34) 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] +(35) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_item_id#27] -(38) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] -(41) HashAggregate [codegen id : 3] +(42) 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 +(43) CometColumnarExchange +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(44) CometColumnarToRow [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] +(45) 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) CometScan parquet spark_catalog.default.web_sales +(46) CometScan 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 +288,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(47) 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] +(48) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(49) 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 +(50) 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] +(51) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(52) 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 +(53) 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] +(54) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_item_id#40] -(53) CometBroadcastHashJoin +(55) 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 +(56) 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] +(57) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] -(56) HashAggregate [codegen id : 5] +(58) 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 +(59) CometColumnarExchange Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(60) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] + +(61) 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 +(62) Union -(60) HashAggregate [codegen id : 7] +(63) 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 +(64) CometColumnarExchange +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(65) CometColumnarToRow [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] +(66) 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 +(67) 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 (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(64) CometScan parquet spark_catalog.default.date_dim +(68) CometScan 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 +(69) 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 +(70) 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) 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 = 30 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 = 46 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 c4ce35e153..d4fb6cd92a 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,93 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + WholeStageCodegen (3) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + WholeStageCodegen (5) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 8423e825d4..9d576a6a59 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,54 @@ == 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 (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (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 (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan parquet spark_catalog.default.item @@ -130,7 +135,7 @@ Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] (19) HashAggregate [codegen id : 1] @@ -140,142 +145,157 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(22) 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 +(23) 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, CometColumnarShuffle, [plan_id=2] + +(24) CometSort 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: [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] -(23) Sort [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(24) Window +(26) 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] +(27) 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 +(28) 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] +(29) 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] +(30) 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] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 6] +(32) CometColumnarToRow [codegen id : 6] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(33) 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 +(34) 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, CometColumnarShuffle, [plan_id=3] + +(35) CometSort 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: [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] -(32) Sort [codegen id : 7] +(36) CometColumnarToRow [codegen id : 7] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 -(33) Window +(37) 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] +(38) 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 +(39) 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] +(40) 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] +(41) 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] +(42) ReusedExchange [Reuses operator id: 34] 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] +(43) 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, 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] + +(44) CometColumnarToRow [codegen id : 11] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 -(40) Window +(45) 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] +(46) 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 +(47) 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] +(48) 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] +(49) 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 +(50) 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 (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(46) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(54) 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 b64c2efdc5..20e9a8ba8a 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,43 @@ 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] + CometColumnarToRow 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 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 +57,15 @@ 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] + CometColumnarToRow 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 + WholeStageCodegen (6) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +73,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] + CometColumnarToRow 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 809315c9e7..169449db81 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,60 @@ == 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 (56) ++- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Filter (21) + : : +- * HashAggregate (20) + : : +- * CometColumnarToRow (19) + : : +- CometColumnarExchange (18) + : : +- * HashAggregate (17) + : : +- * CometColumnarToRow (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 (36) + : +- * Filter (35) + : +- * HashAggregate (34) + : +- * CometColumnarToRow (33) + : +- CometColumnarExchange (32) + : +- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometProject (29) + : +- CometBroadcastHashJoin (28) + : :- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (22) + : : +- ReusedExchange (24) + : +- ReusedExchange (27) + +- BroadcastExchange (53) + +- * Filter (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- CometColumnarExchange (49) + +- * HashAggregate (48) + +- * CometColumnarToRow (47) + +- CometProject (46) + +- CometBroadcastHashJoin (45) + :- CometProject (43) + : +- CometBroadcastHashJoin (42) + : :- CometFilter (40) + : : +- CometScan parquet spark_catalog.default.web_sales (39) + : +- ReusedExchange (41) + +- ReusedExchange (44) (1) CometScan parquet spark_catalog.default.store_sales @@ -101,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: 59] +(10) ReusedExchange [Reuses operator id: 62] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -126,7 +129,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#7], Inner, BuildRight Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#7] Arguments: [ss_ext_sales_price#2, i_item_id#6], [ss_ext_sales_price#2, i_item_id#6] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#2, i_item_id#6] (17) HashAggregate [codegen id : 1] @@ -136,22 +139,25 @@ 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) CometColumnarExchange Input [2]: [i_item_id#6, sum#11] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 6] +(19) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] + +(20) 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] +(21) Filter [codegen id : 6] Input [2]: [item_id#13, ss_item_rev#14] Condition : isnotnull(ss_item_rev#14) -(21) CometScan parquet spark_catalog.default.catalog_sales +(22) CometScan 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 +165,77 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(22) CometFilter +(23) 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] +(24) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#19, i_item_id#20] -(24) CometBroadcastHashJoin +(25) 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 +(26) 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] +(27) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#21] -(27) CometBroadcastHashJoin +(28) 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 +(29) 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] +(30) CometColumnarToRow [codegen id : 2] Input [2]: [cs_ext_sales_price#16, i_item_id#20] -(30) HashAggregate [codegen id : 2] +(31) 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 +(32) CometColumnarExchange +Input [2]: [i_item_id#20, sum#23] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(33) CometColumnarToRow [codegen id : 3] Input [2]: [i_item_id#20, sum#23] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(32) HashAggregate [codegen id : 3] +(34) 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] +(35) Filter [codegen id : 3] Input [2]: [item_id#25, cs_item_rev#26] Condition : isnotnull(cs_item_rev#26) -(34) BroadcastExchange +(36) 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] +(37) 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] +(38) 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) CometScan parquet spark_catalog.default.web_sales +(39) CometScan 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 +243,170 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(38) CometFilter +(40) 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] +(41) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#31, i_item_id#32] -(40) CometBroadcastHashJoin +(42) 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 +(43) 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] +(44) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#33] -(43) CometBroadcastHashJoin +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 4] Input [2]: [ws_ext_sales_price#28, i_item_id#32] -(46) HashAggregate [codegen id : 4] +(48) 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 +(49) 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] -(48) HashAggregate [codegen id : 5] +(50) CometColumnarToRow [codegen id : 5] +Input [2]: [i_item_id#32, sum#35] + +(51) 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] +(52) Filter [codegen id : 5] Input [2]: [item_id#37, ws_item_rev#38] Condition : isnotnull(ws_item_rev#38) -(50) BroadcastExchange +(53) 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] +(54) 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] +(55) 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 +(56) 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) CometScan parquet spark_catalog.default.date_dim +BroadcastExchange (66) ++- * CometColumnarToRow (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometFilter (58) + : +- CometScan parquet spark_catalog.default.date_dim (57) + +- CometBroadcastExchange (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.date_dim (59) + + +(57) CometScan 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 +(58) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(56) CometScan parquet spark_catalog.default.date_dim +(59) CometScan 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 +(60) 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 +(61) CometProject Input [2]: [d_date#9, d_week_seq#43] Arguments: [d_date#9], [d_date#9] -(59) CometBroadcastExchange +(62) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(60) CometBroadcastHashJoin +(63) 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 +(64) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(62) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(63) BroadcastExchange +(66) 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 = 60 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 = 59 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* CometColumnarToRow (70) ++- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) -(64) CometScan parquet spark_catalog.default.date_dim +(67) CometScan 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 +(68) CometFilter Input [2]: [d_date#46, d_week_seq#47] Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) -(66) CometProject +(69) CometProject Input [2]: [d_date#46, d_week_seq#47] Arguments: [d_week_seq#47], [d_week_seq#47] -(67) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [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 = 22 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 = 39 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 0f5a783b94..ac2685dc60 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,88 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #7 + WholeStageCodegen (2) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + WholeStageCodegen (4) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 b471c34359..5941e47b35 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,49 @@ == 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 (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * HashAggregate (12) + : : : +- * CometColumnarToRow (11) + : : : +- CometColumnarExchange (10) + : : : +- * HashAggregate (9) + : : : +- * CometColumnarToRow (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 (16) + : : +- * CometColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (23) + : +- * CometColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.date_dim (19) + +- BroadcastExchange (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- ReusedExchange (26) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.store (29) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) (1) CometScan parquet spark_catalog.default.store_sales @@ -80,7 +82,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] -(8) ColumnarToRow [codegen id : 1] +(8) CometColumnarToRow [codegen id : 1] Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] (9) HashAggregate [codegen id : 1] @@ -90,162 +92,168 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(11) HashAggregate [codegen id : 8] +(11) CometColumnarToRow [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] + +(12) 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) CometScan parquet spark_catalog.default.store +(13) CometScan 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 +(14) CometFilter Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) -(14) ColumnarToRow [codegen id : 2] +(15) CometColumnarToRow [codegen id : 2] Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] -(15) BroadcastExchange +(16) BroadcastExchange Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] -(16) BroadcastHashJoin [codegen id : 8] +(17) 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] +(18) 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) CometScan parquet spark_catalog.default.date_dim +(19) CometScan 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 +(20) CometFilter Input [2]: [d_month_seq#38, d_week_seq#39] Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) -(20) CometProject +(21) 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] +(22) CometColumnarToRow [codegen id : 3] Input [1]: [d_week_seq#39] -(22) BroadcastExchange +(23) 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] +(24) 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] +(25) 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] +(26) ReusedExchange [Reuses operator id: 10] Output [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] -(26) HashAggregate [codegen id : 7] +(27) CometColumnarToRow [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] + +(28) 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) CometScan parquet spark_catalog.default.store +(29) CometScan 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 +(30) 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] +(31) CometColumnarToRow [codegen id : 5] Input [2]: [s_store_sk#68, s_store_id#69] -(30) BroadcastExchange +(32) 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] +(33) 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] +(34) 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) CometScan parquet spark_catalog.default.date_dim +(35) CometScan 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 +(36) 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 +(37) 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] +(38) CometColumnarToRow [codegen id : 6] Input [1]: [d_week_seq#71] -(37) BroadcastExchange +(39) 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] +(40) 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] +(41) 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 +(42) 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] +(43) 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] +(44) 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 +(45) 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 58bae46d1e..c4e6ad2982 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,30 +7,31 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_month_seq,d_week_seq] @@ -43,19 +44,20 @@ 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 + CometColumnarToRow + InputAdapter + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt index 80ba7d7f4f..1097582089 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,46 @@ == 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 (42) ++- * Filter (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * CometColumnarToRow (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 (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * CometColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (31) + +- * Filter (30) + +- * HashAggregate (29) + +- * CometColumnarToRow (28) + +- CometColumnarExchange (27) + +- * HashAggregate (26) + +- * CometColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) CometScan parquet spark_catalog.default.customer_address @@ -129,7 +131,7 @@ 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] +(19) CometColumnarToRow [codegen id : 4] Input [2]: [ca_state#2, ss_item_sk#5] (20) CometScan parquet spark_catalog.default.item @@ -143,7 +145,7 @@ ReadSchema: struct 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] +(22) CometColumnarToRow [codegen id : 3] Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] (23) CometScan parquet spark_catalog.default.item @@ -157,7 +159,7 @@ ReadSchema: struct Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [i_current_price#16, i_category#17] (26) HashAggregate [codegen id : 1] @@ -167,149 +169,155 @@ 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) CometColumnarExchange Input [3]: [i_category#17, sum#20, count#21] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] + +(29) 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] +(30) Filter [codegen id : 2] Input [2]: [avg(i_current_price)#23, i_category#17] Condition : isnotnull(avg(i_current_price)#23) -(30) BroadcastExchange +(31) 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] +(32) 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] +(33) 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 +(34) 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] +(35) 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] +(36) 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] +(37) 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 +(38) CometColumnarExchange +Input [2]: [ca_state#2, count#25] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) CometColumnarToRow [codegen id : 5] Input [2]: [ca_state#2, count#25] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 5] +(40) 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] +(41) Filter [codegen id : 5] Input [2]: [state#27, cnt#28] Condition : (cnt#28 >= 10) -(40) TakeOrderedAndProject +(42) 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 (47) ++- * CometColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) CometScan parquet spark_catalog.default.date_dim +(43) CometScan 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 +(44) 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 +(45) 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] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(47) 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 = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* CometColumnarToRow (54) ++- CometHashAggregate (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) CometScan parquet spark_catalog.default.date_dim +(48) CometScan 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 +(49) 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 +(50) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(49) CometHashAggregate +(51) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(50) CometExchange +(52) CometExchange Input [1]: [d_month_seq#29] Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(51) CometHashAggregate +(53) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(52) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [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 c5bbfc7d58..6f8c9456a2 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,72 +2,74 @@ 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + WholeStageCodegen (4) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [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 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow + 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)] + CometColumnarToRow + 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 - 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #9 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + CometColumnarToRow + 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 09e20ef233..b0568a4804 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,71 @@ == 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 (67) ++- * HashAggregate (66) + +- * CometColumnarToRow (65) + +- CometColumnarExchange (64) + +- * HashAggregate (63) + +- Union (62) + :- * HashAggregate (29) + : +- * CometColumnarToRow (28) + : +- CometColumnarExchange (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (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 (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (34) + : : : +- CometBroadcastHashJoin (33) + : : : :- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (61) + +- * CometColumnarToRow (60) + +- CometColumnarExchange (59) + +- * HashAggregate (58) + +- * CometColumnarToRow (57) + +- CometProject (56) + +- CometBroadcastHashJoin (55) + :- CometProject (53) + : +- CometBroadcastHashJoin (52) + : :- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (48) + : +- ReusedExchange (51) + +- ReusedExchange (54) (1) CometScan parquet spark_catalog.default.store_sales @@ -180,7 +184,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#12] Arguments: [ss_ext_sales_price#3, i_item_id#12], [ss_ext_sales_price#3, i_item_id#12] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_sales_price#3, i_item_id#12] (26) HashAggregate [codegen id : 1] @@ -190,18 +194,21 @@ 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) CometColumnarExchange Input [2]: [i_item_id#12, sum#16] -Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [2]: [i_item_id#12, sum#16] + +(29) 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) CometScan parquet spark_catalog.default.catalog_sales +(30) CometScan 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 +216,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#22), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter +(31) 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] +(32) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#24] -(32) CometBroadcastHashJoin +(33) 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 +(34) 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] +(35) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#25] -(35) CometBroadcastHashJoin +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#26, i_item_id#27] -(38) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 3] Input [2]: [cs_ext_sales_price#21, i_item_id#27] -(41) HashAggregate [codegen id : 3] +(42) 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 +(43) CometColumnarExchange +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(44) CometColumnarToRow [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] +(45) 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) CometScan parquet spark_catalog.default.web_sales +(46) CometScan 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 +288,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter +(47) 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] +(48) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#37] -(47) CometBroadcastHashJoin +(49) 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 +(50) 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] +(51) ReusedExchange [Reuses operator id: 12] Output [1]: [ca_address_sk#38] -(50) CometBroadcastHashJoin +(52) 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 +(53) 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] +(54) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#39, i_item_id#40] -(53) CometBroadcastHashJoin +(55) 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 +(56) 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] +(57) CometColumnarToRow [codegen id : 5] Input [2]: [ws_ext_sales_price#34, i_item_id#40] -(56) HashAggregate [codegen id : 5] +(58) 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 +(59) CometColumnarExchange Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(58) HashAggregate [codegen id : 6] +(60) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#40, sum#42] + +(61) 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 +(62) Union -(60) HashAggregate [codegen id : 7] +(63) 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 +(64) CometColumnarExchange +Input [3]: [i_item_id#12, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_item_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(65) CometColumnarToRow [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] +(66) 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 +(67) 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 (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(64) CometScan parquet spark_catalog.default.date_dim +(68) CometScan 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 +(69) 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 +(70) 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(68) BroadcastExchange +(72) 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 = 30 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 = 46 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 51025cf34f..230e200b37 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,93 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + WholeStageCodegen (3) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #9 + WholeStageCodegen (5) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 098b72016b..9fc01dd45a 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,71 @@ == 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 (67) ++- * BroadcastNestedLoopJoin Inner BuildRight (66) + :- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometColumnarExchange (40) + : +- * HashAggregate (39) + : +- * CometColumnarToRow (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 (65) + +- * HashAggregate (64) + +- * CometColumnarToRow (63) + +- CometColumnarExchange (62) + +- * HashAggregate (61) + +- * CometColumnarToRow (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometProject (53) + : : +- CometBroadcastHashJoin (52) + : : :- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometProject (47) + : : : : +- CometBroadcastHashJoin (46) + : : : : :- CometFilter (44) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (43) + : : : : +- ReusedExchange (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) CometScan parquet spark_catalog.default.store_sales @@ -242,7 +244,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [codegen id : 1] Input [1]: [ss_ext_sales_price#5] (39) HashAggregate [codegen id : 1] @@ -252,18 +254,21 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Aggregate Attributes [1]: [sum#23] Results [1]: [sum#24] -(40) Exchange +(40) CometColumnarExchange Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(41) HashAggregate [codegen id : 4] +(41) CometColumnarToRow [codegen id : 4] +Input [1]: [sum#24] + +(42) 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) CometScan parquet spark_catalog.default.store_sales +(43) CometScan 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 +276,138 @@ 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 6] Output [1]: [s_store_sk#33] -(45) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) ReusedExchange [Reuses operator id: 18] Output [1]: [d_date_sk#34] -(48) CometBroadcastHashJoin +(49) 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 +(50) 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] +(51) ReusedExchange [Reuses operator id: 23] Output [2]: [c_customer_sk#35, c_current_addr_sk#36] -(51) CometBroadcastHashJoin +(52) 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 +(53) 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] +(54) ReusedExchange [Reuses operator id: 29] Output [1]: [ca_address_sk#37] -(54) CometBroadcastHashJoin +(55) 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 +(56) 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] +(57) ReusedExchange [Reuses operator id: 35] Output [1]: [i_item_sk#38] -(57) CometBroadcastHashJoin +(58) 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 +(59) 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] +(60) CometColumnarToRow [codegen id : 2] Input [1]: [ss_ext_sales_price#30] -(60) HashAggregate [codegen id : 2] +(61) 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 +(62) CometColumnarExchange +Input [1]: [sum#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(63) CometColumnarToRow [codegen id : 3] Input [1]: [sum#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] -(62) HashAggregate [codegen id : 3] +(64) 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 +(65) BroadcastExchange Input [1]: [total#42] Arguments: IdentityBroadcastMode, [plan_id=3] -(64) BroadcastNestedLoopJoin [codegen id : 4] +(66) BroadcastNestedLoopJoin [codegen id : 4] Join type: Inner Join condition: None -(65) Project [codegen id : 4] +(67) 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 (72) ++- * CometColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) -(66) CometScan parquet spark_catalog.default.date_dim +(68) CometScan 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 +(69) 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 +(70) 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(70) BroadcastExchange +(72) 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 = 43 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 8d39d9e4ea..b97056f7ae 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,84 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (2) + HashAggregate [ss_ext_sales_price] [sum,sum] + CometColumnarToRow + 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 274be3d0fd..333697aa37 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,33 @@ == 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 (29) ++- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- CometColumnarExchange (26) + +- * HashAggregate (25) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.web_sales @@ -140,7 +141,7 @@ Arguments: [ws_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11, d_date_sk#12] Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] (25) HashAggregate [codegen id : 1] @@ -150,18 +151,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) HashAggregate [codegen id : 2] +(27) CometColumnarToRow [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(28) 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 +(29) 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 b9e35e2825..c4dfa1708d 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,33 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [_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] + CometColumnarToRow + 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 3124ad22e9..15ac30fa0d 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,35 @@ == 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 (31) ++- * Project (30) + +- * Filter (29) + +- Window (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometColumnarExchange (25) + +- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.item @@ -122,7 +124,7 @@ Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] (21) HashAggregate [codegen id : 1] @@ -132,70 +134,76 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarToRow [codegen id : 2] +Input [3]: [i_manager_id#5, d_moy#17, sum#20] + +(24) 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 +(25) 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] + +(26) 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] +(27) CometColumnarToRow [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 +(28) 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] +(29) 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] +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(30) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) 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] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] -(34) BroadcastExchange +(36) 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 e84d3c2a05..aa51c86c25 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,43 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (3) - Sort [i_manager_id] + CometColumnarToRow 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 + WholeStageCodegen (2) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_manager_id,d_moy] #2 + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 a32680a584..29182e99bf 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,46 +1,46 @@ == Physical Plan == -* ColumnarToRow (181) -+- CometSort (180) - +- CometColumnarExchange (179) - +- * Project (178) - +- * SortMergeJoin Inner (177) - :- * Sort (109) - : +- Exchange (108) - : +- * HashAggregate (107) - : +- * HashAggregate (106) - : +- * Project (105) - : +- * BroadcastHashJoin Inner BuildRight (104) - : :- * Project (98) - : : +- * BroadcastHashJoin Inner BuildRight (97) - : : :- * Project (95) - : : : +- * BroadcastHashJoin Inner BuildRight (94) - : : : :- * Project (89) - : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : :- * Project (86) - : : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : : :- * Project (80) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : : : :- * Project (77) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : : : :- * Project (71) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : : : : : : :- * Project (65) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : : : : :- * Project (62) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : : : : : : : : :- * Project (56) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : : : : : : : : :- * Project (53) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : : : : : : : : :- * Project (47) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : : : : : : : : :- * Project (41) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) +* CometColumnarToRow (183) ++- CometSort (182) + +- CometColumnarExchange (181) + +- CometProject (180) + +- CometSortMergeJoin (179) + :- CometSort (110) + : +- CometColumnarExchange (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) + : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) : : : : : : : : : : : : : : : : : +- CometSort (10) : : : : : : : : : : : : : : : : : +- CometExchange (9) : : : : : : : : : : : : : : : : : +- CometProject (8) @@ -51,135 +51,137 @@ : : : : : : : : : : : : : : : : : +- CometProject (6) : : : : : : : : : : : : : : : : : +- CometFilter (5) : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- Exchange (26) - : : : : : : : : : : : : : : : : +- * HashAggregate (25) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (39) - : : : : : : : : : : : : : : +- * ColumnarToRow (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (45) - : : : : : : : : : : : : : +- * ColumnarToRow (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- BroadcastExchange (51) - : : : : : : : : : : : : +- * ColumnarToRow (50) - : : : : : : : : : : : : +- CometFilter (49) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (48) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- BroadcastExchange (60) - : : : : : : : : : : +- * ColumnarToRow (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- BroadcastExchange (69) - : : : : : : : : +- * ColumnarToRow (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- BroadcastExchange (75) - : : : : : : : +- * ColumnarToRow (74) - : : : : : : : +- CometFilter (73) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (72) - : : : : : : +- ReusedExchange (78) - : : : : : +- BroadcastExchange (84) - : : : : : +- * ColumnarToRow (83) - : : : : : +- CometFilter (82) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (81) - : : : : +- ReusedExchange (87) - : : : +- BroadcastExchange (93) - : : : +- * ColumnarToRow (92) - : : : +- CometFilter (91) - : : : +- CometScan parquet spark_catalog.default.income_band (90) - : : +- ReusedExchange (96) - : +- BroadcastExchange (103) - : +- * ColumnarToRow (102) - : +- CometProject (101) - : +- CometFilter (100) - : +- CometScan parquet spark_catalog.default.item (99) - +- * Sort (176) - +- Exchange (175) - +- * HashAggregate (174) - +- * HashAggregate (173) - +- * Project (172) - +- * BroadcastHashJoin Inner BuildRight (171) - :- * Project (169) - : +- * BroadcastHashJoin Inner BuildRight (168) - : :- * Project (166) - : : +- * BroadcastHashJoin Inner BuildRight (165) - : : :- * Project (163) - : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : :- * Project (160) - : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : :- * Project (157) - : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : :- * Project (154) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : :- * Project (151) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : :- * Project (148) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : :- * Project (145) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : :- * Project (142) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : :- * Project (133) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : : : : : : : : : : :- * Project (130) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (129) - : : : : : : : : : : : : : : :- * Project (127) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (126) - : : : : : : : : : : : : : : : :- * ColumnarToRow (120) - : : : : : : : : : : : : : : : : +- CometSort (119) - : : : : : : : : : : : : : : : : +- CometExchange (118) - : : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (116) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (112) - : : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (110) - : : : : : : : : : : : : : : : : +- CometProject (115) - : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (113) - : : : : : : : : : : : : : : : +- * Sort (125) - : : : : : : : : : : : : : : : +- * Project (124) - : : : : : : : : : : : : : : : +- * Filter (123) - : : : : : : : : : : : : : : : +- * HashAggregate (122) - : : : : : : : : : : : : : : : +- ReusedExchange (121) - : : : : : : : : : : : : : : +- ReusedExchange (128) - : : : : : : : : : : : : : +- ReusedExchange (131) - : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : +- ReusedExchange (146) - : : : : : : : +- ReusedExchange (149) - : : : : : : +- ReusedExchange (152) - : : : : : +- ReusedExchange (155) - : : : : +- ReusedExchange (158) - : : : +- ReusedExchange (161) - : : +- ReusedExchange (164) - : +- ReusedExchange (167) - +- ReusedExchange (170) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (27) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (26) + : : : : : : : : : : : : : : : : +- * HashAggregate (25) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (24) + : : : : : : : : : : : : : : : : +- CometProject (23) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) + : : : : : : : : : : : : : : : : :- CometSort (16) + : : : : : : : : : : : : : : : : : +- CometExchange (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- CometSort (21) + : : : : : : : : : : : : : : : : +- CometExchange (20) + : : : : : : : : : : : : : : : : +- CometProject (19) + : : : : : : : : : : : : : : : : +- CometFilter (18) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * CometColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * CometColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * CometColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * CometColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * CometColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * CometColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * CometColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * CometColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * CometColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- CometSort (178) + +- CometColumnarExchange (177) + +- * HashAggregate (176) + +- * HashAggregate (175) + +- * 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) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) + : : : : : : : : : : : : : : :- * Project (129) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (128) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (121) + : : : : : : : : : : : : : : : : +- CometSort (120) + : : : : : : : : : : : : : : : : +- CometExchange (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 (127) + : : : : : : : : : : : : : : : +- * Project (126) + : : : : : : : : : : : : : : : +- * Filter (125) + : : : : : : : : : : : : : : : +- * HashAggregate (124) + : : : : : : : : : : : : : : : +- * CometColumnarToRow (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : : : : +- 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) (1) CometScan parquet spark_catalog.default.store_sales @@ -230,7 +232,7 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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, 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) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [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] (12) CometScan parquet spark_catalog.default.catalog_sales @@ -288,7 +290,7 @@ Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number# 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] +(24) CometColumnarToRow [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] (25) HashAggregate [codegen id : 2] @@ -298,396 +300,399 @@ Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum((( Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(26) Exchange +(26) 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] -(27) HashAggregate [codegen id : 3] +(27) CometColumnarToRow [codegen id : 3] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(28) 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] -(28) Filter [codegen id : 3] +(29) 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))) -(29) Project [codegen id : 3] +(30) Project [codegen id : 3] Output [1]: [cs_item_sk#17] Input [3]: [cs_item_sk#17, sale#35, refund#36] -(30) Sort [codegen id : 3] +(31) Sort [codegen id : 3] Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 19] +(32) SortMergeJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [cs_item_sk#17] Join type: Inner Join condition: None -(32) Project [codegen id : 19] +(33) 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] -(33) ReusedExchange [Reuses operator id: 185] +(34) ReusedExchange [Reuses operator id: 187] Output [2]: [d_date_sk#37, d_year#38] -(34) BroadcastHashJoin [codegen id : 19] +(35) 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 -(35) Project [codegen id : 19] +(36) 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] -(36) CometScan parquet spark_catalog.default.store +(37) CometScan 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 -(37) CometFilter +(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)) -(38) ColumnarToRow [codegen id : 5] +(39) CometColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(39) BroadcastExchange +(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] -(40) BroadcastHashJoin [codegen id : 19] +(41) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(41) Project [codegen id : 19] +(42) 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] -(42) CometScan parquet spark_catalog.default.customer +(43) CometScan 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 -(43) CometFilter +(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)) -(44) ColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [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] -(45) BroadcastExchange +(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] -(46) BroadcastHashJoin [codegen id : 19] +(47) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 19] +(48) 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] -(48) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 -(49) CometFilter +(50) CometFilter Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(50) ColumnarToRow [codegen id : 7] +(51) CometColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] -(51) BroadcastExchange +(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] -(52) BroadcastHashJoin [codegen id : 19] +(53) 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 -(53) Project [codegen id : 19] +(54) 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] -(54) ReusedExchange [Reuses operator id: 51] +(55) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#50, d_year#51] -(55) BroadcastHashJoin [codegen id : 19] +(56) 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 -(56) Project [codegen id : 19] +(57) 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] -(57) CometScan parquet spark_catalog.default.customer_demographics +(58) CometScan 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 -(58) CometFilter +(59) CometFilter Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(59) ColumnarToRow [codegen id : 9] +(60) CometColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(60) BroadcastExchange +(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] -(61) BroadcastHashJoin [codegen id : 19] +(62) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(62) Project [codegen id : 19] +(63) 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] -(63) ReusedExchange [Reuses operator id: 60] +(64) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(64) BroadcastHashJoin [codegen id : 19] +(65) 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) -(65) Project [codegen id : 19] +(66) 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] -(66) CometScan parquet spark_catalog.default.promotion +(67) CometScan 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 -(67) CometFilter +(68) CometFilter Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(68) ColumnarToRow [codegen id : 11] +(69) CometColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] -(69) BroadcastExchange +(70) BroadcastExchange Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(70) BroadcastHashJoin [codegen id : 19] +(71) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(71) Project [codegen id : 19] +(72) 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] -(72) CometScan parquet spark_catalog.default.household_demographics +(73) CometScan 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 -(73) CometFilter +(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)) -(74) ColumnarToRow [codegen id : 12] +(75) CometColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(75) BroadcastExchange +(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] -(76) BroadcastHashJoin [codegen id : 19] +(77) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(77) Project [codegen id : 19] +(78) 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] -(78) ReusedExchange [Reuses operator id: 75] +(79) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(79) BroadcastHashJoin [codegen id : 19] +(80) 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 -(80) Project [codegen id : 19] +(81) 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] -(81) CometScan parquet spark_catalog.default.customer_address +(82) CometScan 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 -(82) CometFilter +(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) -(83) ColumnarToRow [codegen id : 14] +(84) CometColumnarToRow [codegen id : 14] Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(84) BroadcastExchange +(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] -(85) BroadcastHashJoin [codegen id : 19] +(86) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(86) Project [codegen id : 19] +(87) 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] -(87) ReusedExchange [Reuses operator id: 84] +(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] -(88) BroadcastHashJoin [codegen id : 19] +(89) 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 -(89) Project [codegen id : 19] +(90) 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] -(90) CometScan parquet spark_catalog.default.income_band +(91) CometScan 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 -(91) CometFilter +(92) CometFilter Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(92) ColumnarToRow [codegen id : 16] +(93) CometColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] -(93) BroadcastExchange +(94) BroadcastExchange Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(94) BroadcastHashJoin [codegen id : 19] +(95) 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 -(95) Project [codegen id : 19] +(96) 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] -(96) ReusedExchange [Reuses operator id: 93] +(97) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#72] -(97) BroadcastHashJoin [codegen id : 19] +(98) 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 -(98) Project [codegen id : 19] +(99) 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] -(99) CometScan parquet spark_catalog.default.item +(100) CometScan 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 -(100) CometFilter +(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)) -(101) CometProject +(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] -(102) ColumnarToRow [codegen id : 18] +(103) CometColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] -(103) BroadcastExchange +(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] -(104) BroadcastHashJoin [codegen id : 19] +(105) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(105) Project [codegen id : 19] +(106) 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] -(106) HashAggregate [codegen id : 19] +(107) 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] -(107) HashAggregate [codegen id : 19] +(108) 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] -(108) Exchange +(109) 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] -(109) Sort [codegen id : 20] +(110) 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] -(110) CometScan parquet spark_catalog.default.store_sales +(111) CometScan 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 [] @@ -695,368 +700,370 @@ 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 -(111) CometFilter +(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)) -(112) CometBroadcastExchange +(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] -(113) CometScan parquet spark_catalog.default.store_returns +(114) CometScan 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 -(114) CometFilter +(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)) -(115) CometProject +(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] -(116) CometBroadcastHashJoin +(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 -(117) CometProject +(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] -(118) CometExchange +(119) CometExchange 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, CometNativeShuffle, [plan_id=15] -(119) CometSort +(120) 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: [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] -(120) ColumnarToRow [codegen id : 21] +(121) CometColumnarToRow [codegen id : 20] 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] -(121) ReusedExchange [Reuses operator id: 26] +(122) ReusedExchange [Reuses operator id: 26] Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(122) HashAggregate [codegen id : 23] +(123) CometColumnarToRow [codegen id : 22] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(124) HashAggregate [codegen id : 22] 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] -(123) Filter [codegen id : 23] +(125) Filter [codegen id : 22] 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))) -(124) Project [codegen id : 23] +(126) Project [codegen id : 22] Output [1]: [cs_item_sk#122] Input [3]: [cs_item_sk#122, sale#130, refund#131] -(125) Sort [codegen id : 23] +(127) Sort [codegen id : 22] Input [1]: [cs_item_sk#122] Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(126) SortMergeJoin [codegen id : 39] +(128) SortMergeJoin [codegen id : 38] Left keys [1]: [ss_item_sk#106] Right keys [1]: [cs_item_sk#122] Join type: Inner Join condition: None -(127) Project [codegen id : 39] +(129) Project [codegen id : 38] 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] -(128) ReusedExchange [Reuses operator id: 189] +(130) ReusedExchange [Reuses operator id: 191] Output [2]: [d_date_sk#132, d_year#133] -(129) BroadcastHashJoin [codegen id : 39] +(131) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#117] Right keys [1]: [d_date_sk#132] Join type: Inner Join condition: None -(130) Project [codegen id : 39] +(132) Project [codegen id : 38] 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] -(131) ReusedExchange [Reuses operator id: 39] +(133) ReusedExchange [Reuses operator id: 40] Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] -(132) BroadcastHashJoin [codegen id : 39] +(134) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_store_sk#111] Right keys [1]: [s_store_sk#134] Join type: Inner Join condition: None -(133) Project [codegen id : 39] +(135) Project [codegen id : 38] 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] -(134) ReusedExchange [Reuses operator id: 45] +(136) ReusedExchange [Reuses operator id: 46] 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] -(135) BroadcastHashJoin [codegen id : 39] +(137) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_customer_sk#107] Right keys [1]: [c_customer_sk#137] Join type: Inner Join condition: None -(136) Project [codegen id : 39] +(138) Project [codegen id : 38] 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] -(137) ReusedExchange [Reuses operator id: 51] +(139) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#143, d_year#144] -(138) BroadcastHashJoin [codegen id : 39] +(140) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_first_sales_date_sk#142] Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(139) Project [codegen id : 39] +(141) Project [codegen id : 38] 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] -(140) ReusedExchange [Reuses operator id: 51] +(142) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#145, d_year#146] -(141) BroadcastHashJoin [codegen id : 39] +(143) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_first_shipto_date_sk#141] Right keys [1]: [d_date_sk#145] Join type: Inner Join condition: None -(142) Project [codegen id : 39] +(144) Project [codegen id : 38] 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] -(143) ReusedExchange [Reuses operator id: 60] +(145) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(144) BroadcastHashJoin [codegen id : 39] +(146) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_cdemo_sk#108] Right keys [1]: [cd_demo_sk#147] Join type: Inner Join condition: None -(145) Project [codegen id : 39] +(147) Project [codegen id : 38] 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] -(146) ReusedExchange [Reuses operator id: 60] +(148) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#149, cd_marital_status#150] -(147) BroadcastHashJoin [codegen id : 39] +(149) BroadcastHashJoin [codegen id : 38] 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) -(148) Project [codegen id : 39] +(150) Project [codegen id : 38] 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] -(149) ReusedExchange [Reuses operator id: 69] +(151) ReusedExchange [Reuses operator id: 70] Output [1]: [p_promo_sk#151] -(150) BroadcastHashJoin [codegen id : 39] +(152) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_promo_sk#112] Right keys [1]: [p_promo_sk#151] Join type: Inner Join condition: None -(151) Project [codegen id : 39] +(153) Project [codegen id : 38] 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] -(152) ReusedExchange [Reuses operator id: 75] +(154) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(153) BroadcastHashJoin [codegen id : 39] +(155) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_hdemo_sk#109] Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(154) Project [codegen id : 39] +(156) Project [codegen id : 38] 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] -(155) ReusedExchange [Reuses operator id: 75] +(157) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] -(156) BroadcastHashJoin [codegen id : 39] +(158) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_current_hdemo_sk#139] Right keys [1]: [hd_demo_sk#154] Join type: Inner Join condition: None -(157) Project [codegen id : 39] +(159) Project [codegen id : 38] 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] -(158) ReusedExchange [Reuses operator id: 84] +(160) ReusedExchange [Reuses operator id: 85] Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(159) BroadcastHashJoin [codegen id : 39] +(161) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_addr_sk#110] Right keys [1]: [ca_address_sk#156] Join type: Inner Join condition: None -(160) Project [codegen id : 39] +(162) Project [codegen id : 38] 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] -(161) ReusedExchange [Reuses operator id: 84] +(163) ReusedExchange [Reuses operator id: 85] Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(162) BroadcastHashJoin [codegen id : 39] +(164) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_current_addr_sk#140] Right keys [1]: [ca_address_sk#161] Join type: Inner Join condition: None -(163) Project [codegen id : 39] +(165) Project [codegen id : 38] 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] -(164) ReusedExchange [Reuses operator id: 93] +(166) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#166] -(165) BroadcastHashJoin [codegen id : 39] +(167) BroadcastHashJoin [codegen id : 38] Left keys [1]: [hd_income_band_sk#153] Right keys [1]: [ib_income_band_sk#166] Join type: Inner Join condition: None -(166) Project [codegen id : 39] +(168) Project [codegen id : 38] 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] -(167) ReusedExchange [Reuses operator id: 93] +(169) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#167] -(168) BroadcastHashJoin [codegen id : 39] +(170) BroadcastHashJoin [codegen id : 38] Left keys [1]: [hd_income_band_sk#155] Right keys [1]: [ib_income_band_sk#167] Join type: Inner Join condition: None -(169) Project [codegen id : 39] +(171) Project [codegen id : 38] 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] -(170) ReusedExchange [Reuses operator id: 103] +(172) ReusedExchange [Reuses operator id: 104] Output [2]: [i_item_sk#168, i_product_name#169] -(171) BroadcastHashJoin [codegen id : 39] +(173) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#106] Right keys [1]: [i_item_sk#168] Join type: Inner Join condition: None -(172) Project [codegen id : 39] +(174) Project [codegen id : 38] 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] -(173) HashAggregate [codegen id : 39] +(175) HashAggregate [codegen id : 38] 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] -(174) HashAggregate [codegen id : 39] +(176) HashAggregate [codegen id : 38] 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] -(175) Exchange +(177) 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, [plan_id=16] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(176) Sort [codegen id : 40] +(178) CometSort 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 +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) 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) CometSortMergeJoin +Left output [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] +Right output [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#176, store_name#177, store_zip#178], Inner, (cnt#180 <= cnt#102) -(178) 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] +(180) CometProject 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] +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, 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] -(179) CometColumnarExchange +(181) 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] -(180) CometSort +(182) 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: [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) ColumnarToRow [codegen id : 42] +(183) CometColumnarToRow [codegen id : 39] 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] ===== 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 (187) ++- * CometColumnarToRow (186) + +- CometFilter (185) + +- CometScan parquet spark_catalog.default.date_dim (184) -(182) CometScan parquet spark_catalog.default.date_dim +(184) CometScan 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 +(185) 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] +(186) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] -(185) BroadcastExchange +(187) 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 = 110 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 = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (191) ++- * CometColumnarToRow (190) + +- CometFilter (189) + +- CometScan parquet spark_catalog.default.date_dim (188) -(186) CometScan parquet spark_catalog.default.date_dim +(188) CometScan 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 +(189) 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] +(190) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] -(189) BroadcastExchange +(191) 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 1448eff58e..36dafb43b1 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,268 +1,263 @@ -WholeStageCodegen (42) - ColumnarToRow +WholeStageCodegen (39) + CometColumnarToRow InputAdapter 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 - 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) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - 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 - 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] - CometExchange [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] - InputAdapter - Exchange [cs_item_sk] #6 - 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 + CometProject [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] + CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + 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 + 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) + CometColumnarToRow + InputAdapter + 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] + CometExchange [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) + CometColumnarToRow InputAdapter - 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] - CometExchange [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] - CometExchange [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 + 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 - 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #6 + 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] + CometColumnarToRow + InputAdapter + 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] + CometExchange [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] + CometExchange [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 - 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] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (7) - ColumnarToRow + BroadcastExchange #9 + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + 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] #11 + BroadcastExchange #10 + WholeStageCodegen (6) + CometColumnarToRow + 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 #12 - WholeStageCodegen (9) - ColumnarToRow + BroadcastExchange #11 + WholeStageCodegen (7) + CometColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (11) - ColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (9) + CometColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + BroadcastExchange #13 + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - BroadcastExchange #15 - WholeStageCodegen (14) - ColumnarToRow + BroadcastExchange #14 + WholeStageCodegen (12) + CometColumnarToRow 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 [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow + BroadcastExchange #15 + WholeStageCodegen (14) + CometColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_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 [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (18) - ColumnarToRow + BroadcastExchange #16 + WholeStageCodegen (16) + CometColumnarToRow 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 (40) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - 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 - 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] - CometExchange [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] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + 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 (18) + CometColumnarToRow + 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] + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (38) + 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 - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + 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] + CometExchange [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) + CometColumnarToRow + 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 (22) + 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] + CometColumnarToRow + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 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] #21 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + 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 [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + 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 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + 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 e72d488697..dd292d43ae 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,49 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * ColumnarToRow (3) +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * CometColumnarToRow (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 (18) + : : +- * Filter (17) + : : +- * HashAggregate (16) + : : +- * CometColumnarToRow (15) + : : +- CometColumnarExchange (14) + : : +- * HashAggregate (13) + : : +- * CometColumnarToRow (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 (24) + : +- * CometColumnarToRow (23) + : +- CometFilter (22) + : +- CometScan parquet spark_catalog.default.item (21) + +- BroadcastExchange (42) + +- * Filter (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- * CometColumnarToRow (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometFilter (28) + : +- CometScan parquet spark_catalog.default.store_sales (27) + +- ReusedExchange (29) (1) CometScan parquet spark_catalog.default.store @@ -54,7 +57,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) ColumnarToRow [codegen id : 7] +(3) CometColumnarToRow [codegen id : 7] Input [2]: [s_store_sk#1, s_store_name#2] (4) CometScan parquet spark_catalog.default.store_sales @@ -97,7 +100,7 @@ Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(12) ColumnarToRow [codegen id : 1] +(12) CometColumnarToRow [codegen id : 1] Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] (13) HashAggregate [codegen id : 1] @@ -107,64 +110,67 @@ 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) 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, [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] +(15) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#11] + +(16) 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] +(17) Filter [codegen id : 2] Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#13] Condition : isnotnull(revenue#13) -(17) BroadcastExchange +(18) 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] +(19) 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] +(20) 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) CometScan parquet spark_catalog.default.item +(21) CometScan 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 +(22) 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] +(23) CometColumnarToRow [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 +(24) 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] +(25) 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] +(26) 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) CometScan parquet spark_catalog.default.store_sales +(27) CometScan 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 +178,121 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(27) CometFilter +(28) 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] +(29) ReusedExchange [Reuses operator id: 9] Output [1]: [d_date_sk#24] -(29) CometBroadcastHashJoin +(30) 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 +(31) 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] +(32) CometColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -(32) HashAggregate [codegen id : 4] +(33) 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 +(34) 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, CometColumnarShuffle, [plan_id=4] + +(35) CometColumnarToRow [codegen id : 5] Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#26] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(34) HashAggregate [codegen id : 5] +(36) 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] +(37) 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 +(38) CometColumnarExchange +Input [3]: [ss_store_sk#20, sum#31, count#32] +Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(39) CometColumnarToRow [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] +(40) 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] +(41) Filter [codegen id : 6] Input [2]: [ss_store_sk#20, ave#34] Condition : isnotnull(ave#34) -(39) BroadcastExchange +(42) 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] +(43) 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] +(44) 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 +(45) 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 (50) ++- * CometColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) -(43) CometScan parquet spark_catalog.default.date_dim +(46) CometScan 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 +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(47) BroadcastExchange +(50) 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 = 27 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 84d23af42e..775ff36d4c 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 @@ -6,7 +6,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes 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 + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_store_name] CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] @@ -15,32 +15,33 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk,ss_item_sk] #2 + WholeStageCodegen (1) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow 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] @@ -49,20 +50,22 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_store_sk,ss_item_sk] #8 + WholeStageCodegen (4) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + CometColumnarToRow + 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 05f7b272ba..72e3df765a 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,58 @@ == 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 (54) ++- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- Union (49) + :- * HashAggregate (29) + : +- * CometColumnarToRow (28) + : +- CometColumnarExchange (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (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 (48) + +- * CometColumnarToRow (47) + +- CometColumnarExchange (46) + +- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometProject (34) + : : : +- CometBroadcastHashJoin (33) + : : : :- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- ReusedExchange (41) (1) CometScan parquet spark_catalog.default.web_sales @@ -168,7 +171,7 @@ Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#21], Inner, BuildRight Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, sm_ship_mode_sk#21] Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] (26) HashAggregate [codegen id : 1] @@ -178,18 +181,21 @@ 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) 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, [plan_id=1] +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] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [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] + +(29) 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) CometScan parquet spark_catalog.default.catalog_sales +(30) CometScan 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 +203,136 @@ 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 +(31) CometFilter Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) -(31) ReusedExchange [Reuses operator id: 5] +(32) 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 +(33) 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 +(34) 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] +(35) ReusedExchange [Reuses operator id: 10] Output [3]: [d_date_sk#184, d_year#185, d_moy#186] -(35) CometBroadcastHashJoin +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 16] Output [1]: [t_time_sk#187] -(38) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) ReusedExchange [Reuses operator id: 22] Output [1]: [sm_ship_mode_sk#188] -(41) CometBroadcastHashJoin +(42) 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 +(43) 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] +(44) CometColumnarToRow [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] +(45) 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 +(46) 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, CometColumnarShuffle, [plan_id=2] + +(47) CometColumnarToRow [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] -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] -(46) HashAggregate [codegen id : 4] +(48) 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 +(49) Union -(48) HashAggregate [codegen id : 5] +(50) 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 +(51) 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] + +(52) CometColumnarToRow [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] +(53) 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 +(54) 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 (58) ++- * CometColumnarToRow (57) + +- CometFilter (56) + +- CometScan parquet spark_catalog.default.date_dim (55) -(52) CometScan parquet spark_catalog.default.date_dim +(55) CometScan 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 +(56) 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] +(57) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] -(55) BroadcastExchange +(58) 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 = 30 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 880c38bbf8..e19504377a 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,74 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 e07d18f0d2..0b425fbe46 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,37 @@ == 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 (33) ++- * Filter (32) + +- Window (31) + +- WindowGroupLimit (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometColumnarExchange (27) + +- WindowGroupLimit (26) + +- * Sort (25) + +- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -124,7 +126,7 @@ Arguments: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand# Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13] Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, i_product_name#18, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, i_brand#15, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#17, i_class#16, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#17, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#19, i_class#20, i_brand#21, i_product_name#22, d_year#23, d_qoy#24, d_moy#25, s_store_id#26, spark_grouping_id#27] (21) HashAggregate [codegen id : 1] @@ -134,78 +136,84 @@ 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) 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, [plan_id=1] +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] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarToRow [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] + +(24) 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] +(25) 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 +(26) 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 +(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#33] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(27) Sort [codegen id : 3] +(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#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] + +(29) CometColumnarToRow [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 +(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#33] Arguments: [i_category#19], [sumsales#33 DESC NULLS LAST], rank(sumsales#33), 100, Final -(29) Window +(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#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] +(32) 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 +(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#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 (38) ++- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(32) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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) CometColumnarToRow [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-spark4_0/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt index 46d9cf4ef6..010b746363 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,45 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 d27ff35d49..364566ffc2 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,47 @@ == 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 (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * HashAggregate (30) + : : +- * CometColumnarToRow (29) + : : +- CometColumnarExchange (28) + : : +- * HashAggregate (27) + : : +- * CometColumnarToRow (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 (34) + : +- * CometColumnarToRow (33) + : +- CometFilter (32) + : +- CometScan parquet spark_catalog.default.customer (31) + +- BroadcastExchange (40) + +- * CometColumnarToRow (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.customer_address (37) (1) CometScan parquet spark_catalog.default.store_sales @@ -163,7 +164,7 @@ Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] (27) HashAggregate [codegen id : 1] @@ -173,106 +174,109 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(29) HashAggregate [codegen id : 4] +(29) CometColumnarToRow [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] + +(30) 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) CometScan parquet spark_catalog.default.customer +(31) CometScan 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 +(32) 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] +(33) CometColumnarToRow [codegen id : 2] Input [4]: [c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] -(33) BroadcastExchange +(34) 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] +(35) 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] +(36) 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) CometScan parquet spark_catalog.default.customer_address +(37) CometScan 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 +(38) 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] +(39) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#38, ca_city#39] -(39) BroadcastExchange +(40) 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] +(41) 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] +(42) 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 +(43) 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 (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan 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 +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(47) BroadcastExchange +(48) 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 78e1b1ccea..011589b95f 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,56 +5,57 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometFilter [ca_address_sk,ca_city] CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt index 5386a3d166..d1080a5599 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +187,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(50) 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 c9cc4959b7..09ef18a99b 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) - 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_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] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + CometColumnarToRow + 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) + CometColumnarToRow + 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] 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 4dbf5f7751..28bc4d1025 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,35 @@ == 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 (31) ++- * HashAggregate (30) + +- * CometColumnarToRow (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -151,7 +152,7 @@ Arguments: [ss_promo_sk#3], [p_promo_sk#18], Inner, BuildRight Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] (27) HashAggregate [codegen id : 1] @@ -161,50 +162,53 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(29) HashAggregate [codegen id : 2] +(29) CometColumnarToRow [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] + +(30) 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 +(31) 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 (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.date_dim (32) -(31) CometScan parquet spark_catalog.default.date_dim +(32) CometScan 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 +(33) 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 +(34) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(35) BroadcastExchange +(36) 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 a27daca366..38eb1276aa 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,43 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 be8061926c..09e1304637 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,52 @@ == 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 (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Expand (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * CometColumnarToRow (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 (35) + +- * BroadcastHashJoin LeftSemi BuildRight (34) + :- * CometColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (33) + +- * Project (32) + +- * Filter (31) + +- Window (30) + +- WindowGroupLimit (29) + +- * Sort (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- CometColumnarExchange (25) + +- * HashAggregate (24) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -86,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) CometColumnarToRow [codegen id : 5] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -100,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) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -151,7 +154,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#11, s_state#15] (24) HashAggregate [codegen id : 1] @@ -161,132 +164,141 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Aggregate Attributes [1]: [sum#17] Results [2]: [s_state#15, sum#18] -(25) Exchange +(25) CometColumnarExchange Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(26) HashAggregate [codegen id : 2] +(26) CometColumnarToRow [codegen id : 2] +Input [2]: [s_state#15, sum#18] + +(27) 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] +(28) 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 +(29) 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 +(30) 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] +(31) 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] +(32) Project [codegen id : 3] Output [1]: [s_state#15] Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -(32) BroadcastExchange +(33) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(34) BroadcastExchange +(35) 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] +(36) 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] +(37) 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] +(38) 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] +(39) 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 +(40) 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] + +(41) CometColumnarToRow [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] +(42) 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 +(43) 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] + +(44) 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] +(45) CometColumnarToRow [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 +(46) 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] +(47) 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 +(48) 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 (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(46) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 +(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)) -(48) CometProject +(51) 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] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(50) 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-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index 4a6dffc082..9cc7baeeec 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,72 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (7) - Sort [_w1,_w2,_w0] + CometColumnarToRow 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] - 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 #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] - InputAdapter - Exchange [s_state] #7 - 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 + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #7 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + CometColumnarToRow + 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 a36aa81621..8f437ef365 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -1,42 +1,43 @@ == Physical Plan == -* ColumnarToRow (38) -+- CometSort (37) - +- CometColumnarExchange (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) +* CometColumnarToRow (39) ++- CometSort (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.item @@ -188,7 +189,7 @@ Arguments: [time_sk#15], [t_time_sk#34], Inner, BuildRight Input [7]: [i_brand_id#2, i_brand#3, ext_price#13, time_sk#15, t_time_sk#34, t_hour#35, t_minute#36] Arguments: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36], [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [5]: [i_brand_id#2, i_brand#3, ext_price#13, t_hour#35, t_minute#36] (33) HashAggregate [codegen id : 1] @@ -198,57 +199,60 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(35) HashAggregate [codegen id : 2] +(35) CometColumnarToRow [codegen id : 2] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#39] + +(36) 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) CometColumnarExchange +(37) 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] -(37) CometSort +(38) CometSort Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] 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] -(38) ColumnarToRow [codegen id : 3] +(39) CometColumnarToRow [codegen id : 3] Input [5]: [brand_id#41, brand#42, t_hour#35, t_minute#36, ext_price#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (43) -+- * ColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan parquet spark_catalog.default.date_dim (39) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(39) CometScan parquet spark_catalog.default.date_dim +(40) CometScan parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_moy#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter +(41) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11)) AND (d_year#11 = 1999)) AND isnotnull(d_date_sk#10)) -(41) CometProject +(42) CometProject Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(43) BroadcastExchange +(44) 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 1e03b71083..111ef18bf9 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,54 +1,55 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 6aae88fff3..4179f3b1d6 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 b0ca5520f2..a7c93664a7 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 4417365325..eaede7232d 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 70e62b6791..5f5c8bcdf0 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,79 @@ == 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 (75) ++- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * CometColumnarToRow (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 (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.date_dim (26) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * HashAggregate (52) + : +- * CometColumnarToRow (51) + : +- CometColumnarExchange (50) + : +- * HashAggregate (49) + : +- * CometColumnarToRow (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- CometBroadcastExchange (42) + : : +- CometFilter (41) + : : +- CometScan parquet spark_catalog.default.web_sales (40) + : +- ReusedExchange (45) + +- BroadcastExchange (72) + +- * HashAggregate (71) + +- * CometColumnarToRow (70) + +- CometColumnarExchange (69) + +- * HashAggregate (68) + +- * CometColumnarToRow (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.customer (57) + : +- CometBroadcastExchange (61) + : +- CometFilter (60) + : +- CometScan parquet spark_catalog.default.web_sales (59) + +- ReusedExchange (64) (1) CometScan parquet spark_catalog.default.customer @@ -132,7 +136,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] (14) HashAggregate [codegen id : 1] @@ -142,33 +146,36 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarToRow [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(17) 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] +(18) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(18) CometScan parquet spark_catalog.default.customer +(19) CometScan 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 +(20) 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) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +183,93 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(22) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) CometBroadcastExchange +(23) 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 +(24) 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 +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometBroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24, d_year#25] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) CometColumnarToRow [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] +(32) 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 +(33) 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] + +(34) CometColumnarToRow [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] +(35) 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 +(36) 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] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(36) CometScan parquet spark_catalog.default.customer +(38) CometScan 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 +(39) 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) CometScan parquet spark_catalog.default.web_sales +(40) CometScan 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 +277,89 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(41) 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 +(42) 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 +(43) 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) CometColumnarToRow [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] +(49) 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 +(50) 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, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(49) HashAggregate [codegen id : 5] +(51) CometColumnarToRow [codegen id : 5] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(52) 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] +(53) 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 +(54) 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] +(55) 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] +(56) 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) CometScan parquet spark_catalog.default.customer +(57) CometScan 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 +(58) 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) CometScan parquet spark_catalog.default.web_sales +(59) CometScan 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 +367,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(60) 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 +(61) 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 +(62) 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 +(63) 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] +(64) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#55, d_year#56] -(62) CometBroadcastHashJoin +(65) 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 +(66) 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] +(67) CometColumnarToRow [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] +(68) 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 +(69) 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] + +(70) CometColumnarToRow [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] +(71) 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 +(72) 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] +(73) 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] +(74) 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 +(75) 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 (79) ++- * CometColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(72) CometScan parquet spark_catalog.default.date_dim +(76) CometScan 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 +(77) 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] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(79) 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 = 21 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(76) CometScan parquet spark_catalog.default.date_dim +(80) CometScan 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(79) BroadcastExchange +(83) 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 = 40 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 = 59 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 a7aa864ce9..146493bb07 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,100 +7,104 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 66c55173a5..2d34bd004f 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,129 +1,132 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * Project (124) - +- * SortMergeJoin Inner (123) - :- * Sort (67) - : +- Exchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * ColumnarToRow (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (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) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (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) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (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) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.web_returns (50) - +- * Sort (122) - +- Exchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- * ColumnarToRow (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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) +* CometColumnarToRow (128) ++- CometTakeOrderedAndProject (127) + +- CometProject (126) + +- CometSortMergeJoin (125) + :- CometSort (68) + : +- CometColumnarExchange (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (61) + : +- CometHashAggregate (60) + : +- CometExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometExchange (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) + : : +- CometExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometExchange (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) + : : +- CometExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (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) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- CometSort (124) + +- CometColumnarExchange (123) + +- * Filter (122) + +- * HashAggregate (121) + +- * CometColumnarToRow (120) + +- CometColumnarExchange (119) + +- * HashAggregate (118) + +- * CometColumnarToRow (117) + +- CometHashAggregate (116) + +- CometExchange (115) + +- CometHashAggregate (114) + +- CometUnion (113) + :- CometProject (84) + : +- CometSortMergeJoin (83) + : :- CometSort (80) + : : +- CometExchange (79) + : : +- CometProject (78) + : : +- CometBroadcastHashJoin (77) + : : :- CometProject (73) + : : : +- CometBroadcastHashJoin (72) + : : : :- CometFilter (70) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (71) + : : +- CometBroadcastExchange (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.date_dim (74) + : +- CometSort (82) + : +- ReusedExchange (81) + :- CometProject (98) + : +- CometSortMergeJoin (97) + : :- CometSort (94) + : : +- CometExchange (93) + : : +- CometProject (92) + : : +- CometBroadcastHashJoin (91) + : : :- CometProject (89) + : : : +- CometBroadcastHashJoin (88) + : : : :- CometFilter (86) + : : : : +- CometScan parquet spark_catalog.default.store_sales (85) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (90) + : +- CometSort (96) + : +- ReusedExchange (95) + +- CometProject (112) + +- CometSortMergeJoin (111) + :- CometSort (108) + : +- CometExchange (107) + : +- CometProject (106) + : +- CometBroadcastHashJoin (105) + : :- CometProject (103) + : : +- CometBroadcastHashJoin (102) + : : :- CometFilter (100) + : : : +- CometScan parquet spark_catalog.default.web_sales (99) + : : +- ReusedExchange (101) + : +- ReusedExchange (104) + +- CometSort (110) + +- ReusedExchange (109) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -401,7 +404,7 @@ Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_ 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: [] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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] @@ -411,30 +414,33 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) Exchange +(63) 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] -(64) HashAggregate [codegen id : 2] +(64) CometColumnarToRow [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] + +(65) 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] -(65) Filter [codegen id : 2] +(66) 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) -(66) Exchange +(67) 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, [plan_id=9] +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] -(67) Sort [codegen id : 3] +(68) 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: [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 +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] -(68) CometScan parquet spark_catalog.default.catalog_sales +(69) CometScan 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 [] @@ -442,71 +448,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter +(70) 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) -(70) ReusedExchange [Reuses operator id: 6] +(71) 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] -(71) CometBroadcastHashJoin +(72) 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 -(72) CometProject +(73) 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] -(73) CometScan parquet spark_catalog.default.date_dim +(74) CometScan 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 -(74) CometFilter +(75) 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)) -(75) CometBroadcastExchange +(76) CometBroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: [d_date_sk#81, d_year#82] -(76) CometBroadcastHashJoin +(77) 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 -(77) CometProject +(78) 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] -(78) CometExchange +(79) CometExchange 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, CometNativeShuffle, [plan_id=10] -(79) CometSort +(80) 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_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] -(80) ReusedExchange [Reuses operator id: 19] +(81) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(81) CometSort +(82) CometSort Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] 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] -(82) CometSortMergeJoin +(83) 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 -(83) CometProject +(84) 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] -(84) CometScan parquet spark_catalog.default.store_sales +(85) CometScan 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 [] @@ -514,59 +520,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter +(86) 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) -(86) ReusedExchange [Reuses operator id: 6] +(87) 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] -(87) CometBroadcastHashJoin +(88) 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 -(88) CometProject +(89) 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] -(89) ReusedExchange [Reuses operator id: 75] +(90) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#100, d_year#101] -(90) CometBroadcastHashJoin +(91) 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 -(91) CometProject +(92) 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] -(92) CometExchange +(93) CometExchange 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, CometNativeShuffle, [plan_id=11] -(93) CometSort +(94) 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: [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] -(94) ReusedExchange [Reuses operator id: 36] +(95) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(95) CometSort +(96) CometSort Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] 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] -(96) CometSortMergeJoin +(97) 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 -(97) CometProject +(98) 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] -(98) CometScan parquet spark_catalog.default.web_sales +(99) CometScan 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 [] @@ -574,148 +580,153 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter +(100) 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) -(100) ReusedExchange [Reuses operator id: 6] +(101) 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] -(101) CometBroadcastHashJoin +(102) 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 -(102) CometProject +(103) 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] -(103) ReusedExchange [Reuses operator id: 75] +(104) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#119, d_year#120] -(104) CometBroadcastHashJoin +(105) 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 -(105) CometProject +(106) 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] -(106) CometExchange +(107) CometExchange 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, CometNativeShuffle, [plan_id=12] -(107) CometSort +(108) 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_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] -(108) ReusedExchange [Reuses operator id: 53] +(109) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(109) CometSort +(110) CometSort Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] 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] -(110) CometSortMergeJoin +(111) 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 -(111) CometProject +(112) 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] -(112) CometUnion +(113) 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] -(113) CometHashAggregate +(114) 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: [] -(114) CometExchange +(115) CometExchange 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, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate +(116) 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: [] -(116) ColumnarToRow [codegen id : 4] +(117) CometColumnarToRow [codegen id : 3] 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] -(117) HashAggregate [codegen id : 4] +(118) HashAggregate [codegen id : 3] 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] -(118) Exchange +(119) 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] + +(120) CometColumnarToRow [codegen id : 4] 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] -(119) HashAggregate [codegen id : 5] +(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, 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] -(120) Filter [codegen id : 5] +(122) Filter [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#131, sales_amt#132] Condition : isnotnull(sales_cnt#131) -(121) Exchange +(123) 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, [plan_id=15] +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] -(122) Sort [codegen id : 6] +(124) 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: [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 +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] -(123) 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) +(125) CometSortMergeJoin +Left output [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] +Right output [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#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], Inner, ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) -(124) 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] +(126) CometProject 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] +Arguments: [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], [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] + +(127) CometTakeOrderedAndProject +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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#137 ASC NULLS FIRST], output=[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]), [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], 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] -(125) TakeOrderedAndProject +(128) CometColumnarToRow [codegen id : 5] 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 (129) -+- * ColumnarToRow (128) - +- CometFilter (127) - +- CometScan parquet spark_catalog.default.date_dim (126) +BroadcastExchange (132) ++- * CometColumnarToRow (131) + +- CometFilter (130) + +- CometScan parquet spark_catalog.default.date_dim (129) -(126) CometScan parquet spark_catalog.default.date_dim +(129) CometScan 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 -(127) CometFilter +(130) 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)) -(128) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(129) BroadcastExchange +(132) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] @@ -723,33 +734,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (136) ++- * CometColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) CometScan parquet spark_catalog.default.date_dim +(133) CometScan 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 -(131) CometFilter +(134) 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)) -(132) ColumnarToRow [codegen id : 1] +(135) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(133) BroadcastExchange +(136) 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 = 84 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 99 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 e52b2f7d8a..1c136ee8e3 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,20 +1,20 @@ -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 (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 (3) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (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] +WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,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 + 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] + CometColumnarToRow InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 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 + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 @@ -33,7 +33,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -85,19 +85,17 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat 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 (6) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - 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] + 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 + WholeStageCodegen (4) + 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] + CometColumnarToRow InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - WholeStageCodegen (4) + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (3) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 @@ -116,7 +114,7 @@ TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_cat SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt index 8bfcb729b9..aed75047f1 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,39 @@ == 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 (35) ++- * HashAggregate (34) + +- * CometColumnarToRow (33) + +- CometColumnarExchange (32) + +- * HashAggregate (31) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -172,7 +173,7 @@ Child 0 Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ex Child 1 Input [6]: [channel#22, col_name#23, d_year#20, d_qoy#21, i_category#18, ext_sales_price#24] Child 2 Input [6]: [channel#34, col_name#35, d_year#32, d_qoy#33, i_category#30, ext_sales_price#36] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] (31) HashAggregate [codegen id : 1] @@ -182,18 +183,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(33) HashAggregate [codegen id : 2] +(33) CometColumnarToRow [codegen id : 2] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] + +(34) 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 +(35) 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 cfcf6a99ab..79ffc346f5 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,39 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 210be21218..5dd7869aeb 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,97 @@ == 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 (93) ++- * HashAggregate (92) + +- * CometColumnarToRow (91) + +- CometColumnarExchange (90) + +- * HashAggregate (89) + +- * Expand (88) + +- Union (87) + :- * Project (34) + : +- * BroadcastHashJoin LeftOuter BuildRight (33) + : :- * HashAggregate (18) + : : +- * CometColumnarToRow (17) + : : +- CometColumnarExchange (16) + : : +- * HashAggregate (15) + : : +- * CometColumnarToRow (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 (32) + : +- * HashAggregate (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * CometColumnarToRow (27) + : +- CometProject (26) + : +- CometBroadcastHashJoin (25) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometFilter (20) + : : : +- CometScan parquet spark_catalog.default.store_returns (19) + : : +- ReusedExchange (21) + : +- ReusedExchange (24) + :- * Project (55) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (54) + : :- BroadcastExchange (44) + : : +- * HashAggregate (43) + : : +- * CometColumnarToRow (42) + : : +- CometColumnarExchange (41) + : : +- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometScan parquet spark_catalog.default.catalog_sales (35) + : : +- ReusedExchange (36) + : +- * HashAggregate (53) + : +- * CometColumnarToRow (52) + : +- CometColumnarExchange (51) + : +- * HashAggregate (50) + : +- * CometColumnarToRow (49) + : +- CometProject (48) + : +- CometBroadcastHashJoin (47) + : :- CometScan parquet spark_catalog.default.catalog_returns (45) + : +- ReusedExchange (46) + +- * Project (86) + +- * BroadcastHashJoin LeftOuter BuildRight (85) + :- * HashAggregate (70) + : +- * CometColumnarToRow (69) + : +- CometColumnarExchange (68) + : +- * HashAggregate (67) + : +- * CometColumnarToRow (66) + : +- CometProject (65) + : +- CometBroadcastHashJoin (64) + : :- CometProject (60) + : : +- CometBroadcastHashJoin (59) + : : :- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.web_sales (56) + : : +- ReusedExchange (58) + : +- CometBroadcastExchange (63) + : +- CometFilter (62) + : +- CometScan parquet spark_catalog.default.web_page (61) + +- BroadcastExchange (84) + +- * HashAggregate (83) + +- * CometColumnarToRow (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- * CometColumnarToRow (79) + +- CometProject (78) + +- CometBroadcastHashJoin (77) + :- CometProject (75) + : +- CometBroadcastHashJoin (74) + : :- CometFilter (72) + : : +- CometScan parquet spark_catalog.default.web_returns (71) + : +- ReusedExchange (73) + +- ReusedExchange (76) (1) CometScan parquet spark_catalog.default.store_sales @@ -151,7 +158,7 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] (15) HashAggregate [codegen id : 1] @@ -161,18 +168,21 @@ 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) CometColumnarExchange Input [3]: [s_store_sk#8, sum#11, sum#12] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] + +(18) 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) CometScan parquet spark_catalog.default.store_returns +(19) CometScan 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 +190,171 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(20) 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] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#22] -(21) CometBroadcastHashJoin +(22) 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 +(23) 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] +(24) ReusedExchange [Reuses operator id: 11] Output [1]: [s_store_sk#23] -(24) CometBroadcastHashJoin +(25) 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 +(26) 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] +(27) CometColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(27) HashAggregate [codegen id : 2] +(28) 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 +(29) CometColumnarExchange +Input [3]: [s_store_sk#23, sum#26, sum#27] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(30) CometColumnarToRow [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] +(31) 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 +(32) 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] +(33) 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] +(34) 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) CometScan parquet spark_catalog.default.catalog_sales +(35) CometScan 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] +(36) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#41] -(35) CometBroadcastHashJoin +(37) 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 +(38) 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] +(39) CometColumnarToRow [codegen id : 5] Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] -(38) HashAggregate [codegen id : 5] +(40) 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 +(41) 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] +(42) CometColumnarToRow [codegen id : 6] +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(43) 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 +(44) BroadcastExchange Input [3]: [cs_call_center_sk#36, sales#48, profit#49] Arguments: IdentityBroadcastMode, [plan_id=5] -(42) CometScan parquet spark_catalog.default.catalog_returns +(45) CometScan 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] +(46) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#54] -(44) CometBroadcastHashJoin +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] -(47) HashAggregate [codegen id : 7] +(50) 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 +(51) CometColumnarExchange +Input [2]: [sum#57, sum#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(52) CometColumnarToRow Input [2]: [sum#57, sum#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate +(53) 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] +(54) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(51) Project [codegen id : 8] +(55) 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) CometScan parquet spark_catalog.default.web_sales +(56) CometScan 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 +362,71 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter +(57) 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] +(58) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#71] -(55) CometBroadcastHashJoin +(59) 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 +(60) 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) CometScan parquet spark_catalog.default.web_page +(61) CometScan 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 +(62) CometFilter Input [1]: [wp_web_page_sk#72] Condition : isnotnull(wp_web_page_sk#72) -(59) CometBroadcastExchange +(63) CometBroadcastExchange Input [1]: [wp_web_page_sk#72] Arguments: [wp_web_page_sk#72] -(60) CometBroadcastHashJoin +(64) 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 +(65) 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] +(66) CometColumnarToRow [codegen id : 9] Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] -(63) HashAggregate [codegen id : 9] +(67) 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 +(68) CometColumnarExchange +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(69) CometColumnarToRow [codegen id : 12] Input [3]: [wp_web_page_sk#72, sum#75, sum#76] -Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 12] +(70) 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) CometScan parquet spark_catalog.default.web_returns +(71) CometScan 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 +434,143 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter +(72) 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] +(73) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#86] -(69) CometBroadcastHashJoin +(74) 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 +(75) 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] +(76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#87] -(72) CometBroadcastHashJoin +(77) 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 +(78) 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] +(79) CometColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(75) HashAggregate [codegen id : 10] +(80) 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 +(81) CometColumnarExchange Input [3]: [wp_web_page_sk#87, sum#90, sum#91] -Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(77) HashAggregate [codegen id : 11] +(82) CometColumnarToRow [codegen id : 11] +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(83) 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 +(84) 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] +(85) 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] +(86) 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 +(87) Union -(82) Expand [codegen id : 13] +(88) 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] +(89) 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 +(90) 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] + +(91) CometColumnarToRow [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] +(92) 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 +(93) 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 (98) ++- * CometColumnarToRow (97) + +- CometProject (96) + +- CometFilter (95) + +- CometScan parquet spark_catalog.default.date_dim (94) -(87) CometScan parquet spark_catalog.default.date_dim +(94) CometScan 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 +(95) 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 +(96) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(90) ColumnarToRow [codegen id : 1] +(97) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(91) BroadcastExchange +(98) 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 = 19 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 = 35 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 = 45 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 = 56 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 = 71 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 91786374e3..482144a769 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,137 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #2 + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - Exchange [s_store_sk] #2 - WholeStageCodegen (1) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - ColumnarToRow + 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] + CometColumnarToRow 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] - 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] - 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] + CometColumnarExchange [s_store_sk] #7 + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow + 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 - Exchange #10 - WholeStageCodegen (7) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + 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] + CometColumnarToRow 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] + CometColumnarExchange [cs_call_center_sk] #9 + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #11 + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + CometColumnarToRow + 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 - 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 + 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] + CometColumnarToRow 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] - 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] #14 + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow + 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 b798add795..51b698fd6b 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,73 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (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 (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * ColumnarToRow (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * ColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (58) +TakeOrderedAndProject (72) ++- * Project (71) + +- * SortMergeJoin Inner (70) + :- * Project (47) + : +- * SortMergeJoin Inner (46) + : :- * Sort (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (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 (45) + : +- * Filter (44) + : +- * HashAggregate (43) + : +- * CometColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometFilter (34) + : : +- CometSortMergeJoin (33) + : : :- CometSort (27) + : : : +- CometExchange (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.web_sales (24) + : : +- CometSort (32) + : : +- CometExchange (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.web_returns (28) + : +- ReusedExchange (36) + +- * Sort (69) + +- * Filter (68) + +- * HashAggregate (67) + +- * CometColumnarToRow (66) + +- CometColumnarExchange (65) + +- * HashAggregate (64) + +- * CometColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometProject (59) + : +- CometFilter (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometExchange (50) + : : +- CometFilter (49) + : : +- CometScan parquet spark_catalog.default.catalog_sales (48) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan parquet spark_catalog.default.catalog_returns (52) + +- ReusedExchange (60) (1) CometScan parquet spark_catalog.default.store_sales @@ -150,7 +153,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight 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] +(18) CometColumnarToRow [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] (19) HashAggregate [codegen id : 1] @@ -160,22 +163,25 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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] -(20) Exchange +(20) 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, [plan_id=3] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [codegen id : 2] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(22) 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] -(22) Sort [codegen id : 2] +(23) 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 -(23) CometScan parquet spark_catalog.default.web_sales +(24) CometScan 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 [] @@ -183,106 +189,109 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter +(25) 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)) -(25) CometExchange +(26) CometExchange 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, CometNativeShuffle, [plan_id=4] -(26) CometSort +(27) 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_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) CometScan parquet spark_catalog.default.web_returns +(28) CometScan 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 +(29) 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 +(30) 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) CometExchange +(31) CometExchange Input [2]: [wr_item_sk#35, wr_order_number#36] Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(31) CometSort +(32) CometSort Input [2]: [wr_item_sk#35, wr_order_number#36] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(32) CometSortMergeJoin +(33) 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 -(33) CometFilter +(34) 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) -(34) CometProject +(35) 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] -(35) ReusedExchange [Reuses operator id: 15] +(36) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#38, d_year#39] -(36) CometBroadcastHashJoin +(37) 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 -(37) CometProject +(38) 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] -(38) ColumnarToRow [codegen id : 3] +(39) CometColumnarToRow [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 : 3] +(40) 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 +(41) 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, CometColumnarShuffle, [plan_id=6] + +(42) CometColumnarToRow [codegen id : 4] 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 : 4] +(43) 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 : 4] +(44) 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 : 4] +(45) 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 : 5] +(46) 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 : 5] +(47) 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) CometScan parquet spark_catalog.default.catalog_sales +(48) CometScan 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 [] @@ -290,138 +299,141 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter +(49) 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) CometExchange +(50) CometExchange 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, CometNativeShuffle, [plan_id=7] -(49) CometSort +(51) 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_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] -(50) CometScan parquet spark_catalog.default.catalog_returns +(52) CometScan 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) CometExchange +(55) CometExchange Input [2]: [cr_item_sk#62, cr_order_number#63] Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(54) CometSort +(56) CometSort Input [2]: [cr_item_sk#62, cr_order_number#63] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(55) CometSortMergeJoin +(57) 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 -(56) CometFilter +(58) 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) -(57) CometProject +(59) 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] -(58) ReusedExchange [Reuses operator id: 15] +(60) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#65, d_year#66] -(59) CometBroadcastHashJoin +(61) 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 -(60) CometProject +(62) 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] -(61) ColumnarToRow [codegen id : 6] +(63) CometColumnarToRow [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] -(62) HashAggregate [codegen id : 6] +(64) 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] -(63) Exchange +(65) 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] + +(66) CometColumnarToRow [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] -(64) HashAggregate [codegen id : 7] +(67) 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] -(65) Filter [codegen id : 7] +(68) 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) -(66) Sort [codegen id : 7] +(69) 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 -(67) SortMergeJoin [codegen id : 8] +(70) 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 -(68) Project [codegen id : 8] +(71) Project [codegen id : 8] Output [12]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(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] -(69) TakeOrderedAndProject +(72) 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 (73) -+- * ColumnarToRow (72) - +- CometFilter (71) - +- CometScan parquet spark_catalog.default.date_dim (70) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(70) CometScan parquet spark_catalog.default.date_dim +(73) CometScan 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 -(71) CometFilter +(74) 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)) -(72) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) BroadcastExchange +(76) 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 = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 24 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 = 48 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 48eeb00d60..317761b03d 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 @@ -10,87 +10,90 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes 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 (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] - CometExchange [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] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + 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] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 (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] #6 - 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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 (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] #9 - 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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 9fa6bbb19c..f161935772 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,36 @@ == 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 (32) ++- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * HashAggregate (25) + : +- * CometColumnarToRow (24) + : +- CometColumnarExchange (23) + : +- * HashAggregate (22) + : +- * CometColumnarToRow (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) + +- * CometColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.customer (26) (1) CometScan parquet spark_catalog.default.store_sales @@ -128,7 +129,7 @@ Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] (22) HashAggregate [codegen id : 1] @@ -138,78 +139,81 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(24) HashAggregate [codegen id : 3] +(24) CometColumnarToRow [codegen id : 3] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#21, sum#22] + +(25) 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) CometScan parquet spark_catalog.default.customer +(26) CometScan 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 2] Input [3]: [c_customer_sk#27, c_first_name#28, c_last_name#29] -(28) BroadcastExchange +(29) 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] +(30) 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] +(31) 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 +(32) 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 (37) ++- * CometColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) -(32) CometScan parquet spark_catalog.default.date_dim +(33) CometScan 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 +(34) 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 +(35) 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] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(36) BroadcastExchange +(37) 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 f2883e4503..1346ff91b8 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,44 +3,45 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow 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] 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 99e0e2f16a..de0a10bf04 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,46 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * HashAggregate (41) - +- Exchange (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) - +- CometExchange (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) - +- CometExchange (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) +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (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) + +- CometExchange (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) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -218,7 +219,7 @@ Arguments: [substr(s_zip#10, 1, 2)], [substr(ca_zip#12, 1, 2)], Inner, BuildRigh 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] +(38) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#2, s_store_name#9] (39) HashAggregate [codegen id : 1] @@ -228,50 +229,53 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [2]: [s_store_name#9, sum#21] -(40) Exchange +(40) CometColumnarExchange Input [2]: [s_store_name#9, sum#21] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(41) HashAggregate [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] +Input [2]: [s_store_name#9, sum#21] + +(42) 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))#22] Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] -(42) TakeOrderedAndProject +(43) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan 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 -(44) CometFilter +(45) 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)) -(45) CometProject +(46) CometProject Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(47) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index 150dc4c172..93ef68b5c6 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,54 +1,55 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] WholeStageCodegen (2) HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - InputAdapter - Exchange [s_store_name] #1 - WholeStageCodegen (1) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - ColumnarToRow - InputAdapter - 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] - CometExchange [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)] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_name] #1 + WholeStageCodegen (1) + HashAggregate [s_store_name,ss_net_profit] [sum,sum] + CometColumnarToRow + InputAdapter + 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) + CometColumnarToRow + 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] + CometExchange [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)] + CometExchange [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 63109f7d7a..e31cfc1fcc 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,106 +1,110 @@ == Physical Plan == -TakeOrderedAndProject (102) -+- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Expand (98) - +- Union (97) - :- * HashAggregate (38) - : +- Exchange (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) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (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) - : +- Exchange (66) - : +- * HashAggregate (65) - : +- * ColumnarToRow (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (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) - +- Exchange (95) - +- * HashAggregate (94) - +- * ColumnarToRow (93) - +- CometProject (92) - +- CometBroadcastHashJoin (91) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (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) +TakeOrderedAndProject (106) ++- * HashAggregate (105) + +- * CometColumnarToRow (104) + +- CometColumnarExchange (103) + +- * HashAggregate (102) + +- * Expand (101) + +- Union (100) + :- * HashAggregate (39) + : +- * CometColumnarToRow (38) + : +- CometColumnarExchange (37) + : +- * HashAggregate (36) + : +- * CometColumnarToRow (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (28) + : : +- CometBroadcastHashJoin (27) + : : :- CometProject (22) + : : : +- CometBroadcastHashJoin (21) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (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 (69) + : +- * CometColumnarToRow (68) + : +- CometColumnarExchange (67) + : +- * HashAggregate (66) + : +- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometProject (53) + : : : : +- CometBroadcastHashJoin (52) + : : : : :- CometProject (50) + : : : : : +- CometSortMergeJoin (49) + : : : : : :- CometSort (43) + : : : : : : +- CometExchange (42) + : : : : : : +- CometFilter (41) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : +- CometSort (48) + : : : : : +- CometExchange (47) + : : : : : +- CometProject (46) + : : : : : +- CometFilter (45) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (44) + : : : : +- ReusedExchange (51) + : : : +- CometBroadcastExchange (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.catalog_page (54) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- * HashAggregate (99) + +- * CometColumnarToRow (98) + +- CometColumnarExchange (97) + +- * HashAggregate (96) + +- * CometColumnarToRow (95) + +- CometProject (94) + +- CometBroadcastHashJoin (93) + :- CometProject (91) + : +- CometBroadcastHashJoin (90) + : :- CometProject (88) + : : +- CometBroadcastHashJoin (87) + : : :- CometProject (83) + : : : +- CometBroadcastHashJoin (82) + : : : :- CometProject (80) + : : : : +- CometSortMergeJoin (79) + : : : : :- CometSort (73) + : : : : : +- CometExchange (72) + : : : : : +- CometFilter (71) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (70) + : : : : +- CometSort (78) + : : : : +- CometExchange (77) + : : : : +- CometProject (76) + : : : : +- CometFilter (75) + : : : : +- CometScan parquet spark_catalog.default.web_returns (74) + : : : +- ReusedExchange (81) + : : +- CometBroadcastExchange (86) + : : +- CometFilter (85) + : : +- CometScan parquet spark_catalog.default.web_site (84) + : +- ReusedExchange (89) + +- ReusedExchange (92) (1) CometScan parquet spark_catalog.default.store_sales @@ -263,7 +267,7 @@ Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight 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] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [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] (36) HashAggregate [codegen id : 1] @@ -273,18 +277,21 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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) Exchange +(37) 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, [plan_id=3] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(38) HashAggregate [codegen id : 2] +(38) CometColumnarToRow [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(39) 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))#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] -(39) CometScan parquet spark_catalog.default.catalog_sales +(40) CometScan 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 [] @@ -292,132 +299,135 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter +(41) 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)) -(41) CometExchange +(42) CometExchange 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, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) 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] -(43) CometScan parquet spark_catalog.default.catalog_returns +(44) CometScan parquet spark_catalog.default.catalog_returns 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 -(44) CometFilter +(45) CometFilter 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)) -(45) CometProject +(46) CometProject 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] -(46) CometExchange +(47) CometExchange 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, CometNativeShuffle, [plan_id=5] -(47) CometSort +(48) 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] -(48) CometSortMergeJoin +(49) 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 -(49) CometProject +(50) 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] -(50) ReusedExchange [Reuses operator id: 15] +(51) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#53] -(51) CometBroadcastHashJoin +(52) 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 -(52) CometProject +(53) 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] -(53) CometScan parquet spark_catalog.default.catalog_page +(54) CometScan 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 -(54) CometFilter +(55) CometFilter Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Condition : isnotnull(cp_catalog_page_sk#54) -(55) CometBroadcastExchange +(56) CometBroadcastExchange Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -(56) CometBroadcastHashJoin +(57) 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 -(57) CometProject +(58) 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] -(58) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#56] -(59) CometBroadcastHashJoin +(60) 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 -(60) CometProject +(61) 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] -(61) ReusedExchange [Reuses operator id: 32] +(62) ReusedExchange [Reuses operator id: 32] Output [1]: [p_promo_sk#57] -(62) CometBroadcastHashJoin +(63) 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 -(63) CometProject +(64) 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] -(64) ColumnarToRow [codegen id : 3] +(65) CometColumnarToRow [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] -(65) HashAggregate [codegen id : 3] +(66) 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] -(66) Exchange +(67) 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] + +(68) CometColumnarToRow [codegen id : 4] 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, [plan_id=6] -(67) HashAggregate [codegen id : 4] +(69) 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] -(68) CometScan parquet spark_catalog.default.web_sales +(70) CometScan 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 [] @@ -425,193 +435,199 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(69) CometFilter +(71) 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)) -(70) CometExchange +(72) CometExchange 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, CometNativeShuffle, [plan_id=7] -(71) CometSort +(73) 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] -(72) CometScan parquet spark_catalog.default.web_returns +(74) CometScan parquet spark_catalog.default.web_returns 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 -(73) CometFilter +(75) CometFilter 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)) -(74) CometProject +(76) CometProject 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] -(75) CometExchange +(77) CometExchange 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, CometNativeShuffle, [plan_id=8] -(76) CometSort +(78) 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] -(77) CometSortMergeJoin +(79) 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 -(78) CometProject +(80) 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] -(79) ReusedExchange [Reuses operator id: 15] +(81) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#89] -(80) CometBroadcastHashJoin +(82) 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 -(81) CometProject +(83) 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] -(82) CometScan parquet spark_catalog.default.web_site +(84) CometScan 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 -(83) CometFilter +(85) CometFilter Input [2]: [web_site_sk#90, web_site_id#91] Condition : isnotnull(web_site_sk#90) -(84) CometBroadcastExchange +(86) CometBroadcastExchange Input [2]: [web_site_sk#90, web_site_id#91] Arguments: [web_site_sk#90, web_site_id#91] -(85) CometBroadcastHashJoin +(87) 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 -(86) CometProject +(88) 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] -(87) ReusedExchange [Reuses operator id: 26] +(89) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#92] -(88) CometBroadcastHashJoin +(90) 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 -(89) CometProject +(91) 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] -(90) ReusedExchange [Reuses operator id: 32] +(92) ReusedExchange [Reuses operator id: 32] Output [1]: [p_promo_sk#93] -(91) CometBroadcastHashJoin +(93) 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 -(92) CometProject +(94) 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] -(93) ColumnarToRow [codegen id : 5] +(95) CometColumnarToRow [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] -(94) HashAggregate [codegen id : 5] +(96) 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] -(95) Exchange +(97) 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, [plan_id=9] +Arguments: hashpartitioning(web_site_id#91, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(96) HashAggregate [codegen id : 6] +(98) CometColumnarToRow [codegen id : 6] +Input [6]: [web_site_id#91, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] + +(99) 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] -(97) Union +(100) Union -(98) Expand [codegen id : 7] +(101) 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] -(99) HashAggregate [codegen id : 7] +(102) 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] -(100) Exchange +(103) 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] + +(104) CometColumnarToRow [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] -Arguments: hashpartitioning(channel#112, id#113, spark_grouping_id#114, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(101) HashAggregate [codegen id : 8] +(105) 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] -(102) TakeOrderedAndProject +(106) 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 (107) -+- * ColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan parquet spark_catalog.default.date_dim (103) +BroadcastExchange (111) ++- * CometColumnarToRow (110) + +- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(103) CometScan parquet spark_catalog.default.date_dim +(107) CometScan 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 -(104) CometFilter +(108) 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)) -(105) CometProject +(109) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(110) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(107) BroadcastExchange +(111) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 70 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 717c94e212..dfa0a8cb1e 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,128 +1,132 @@ 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,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 (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] - CometExchange [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] - CometExchange [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 (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] - InputAdapter - Exchange [cp_catalog_page_id] #10 - 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] - CometExchange [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] - CometExchange [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 (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] - InputAdapter - Exchange [web_site_id] #14 - 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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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,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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #2 + 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] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #10 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #14 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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-spark4_0/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt index 6b252f661c..1964c2ec09 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,57 @@ == 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 (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) + : : :- * Filter (19) + : : : +- * HashAggregate (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- * HashAggregate (15) + : : : +- * CometColumnarToRow (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 (38) + : : +- * Filter (37) + : : +- * HashAggregate (36) + : : +- * CometColumnarToRow (35) + : : +- CometColumnarExchange (34) + : : +- * HashAggregate (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometColumnarExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometProject (24) + : : : +- CometBroadcastHashJoin (23) + : : : :- CometFilter (21) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (20) + : : : +- ReusedExchange (22) + : : +- ReusedExchange (25) + : +- BroadcastExchange (44) + : +- * CometColumnarToRow (43) + : +- CometFilter (42) + : +- CometScan parquet spark_catalog.default.customer (41) + +- BroadcastExchange (50) + +- * CometColumnarToRow (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.customer_address (47) (1) CometScan parquet spark_catalog.default.catalog_returns @@ -115,7 +118,7 @@ Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight 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] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] (15) HashAggregate [codegen id : 1] @@ -125,22 +128,25 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 7] +(17) CometColumnarToRow [codegen id : 7] +Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] + +(18) 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] +(19) Filter [codegen id : 7] Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) -(19) CometScan parquet spark_catalog.default.catalog_returns +(20) CometScan 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 +154,189 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(20) CometFilter +(21) 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] +(22) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#21] -(22) CometBroadcastHashJoin +(23) 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 +(24) 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] +(25) ReusedExchange [Reuses operator id: 11] Output [2]: [ca_address_sk#22, ca_state#23] -(25) CometBroadcastHashJoin +(26) 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 +(27) 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] +(28) CometColumnarToRow [codegen id : 2] Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] -(28) HashAggregate [codegen id : 2] +(29) 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 +(30) 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] + +(31) CometColumnarToRow [codegen id : 3] 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) 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] +(33) 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 +(34) CometColumnarExchange +Input [3]: [ctr_state#26, sum#30, count#31] +Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(35) CometColumnarToRow [codegen id : 4] Input [3]: [ctr_state#26, sum#30, count#31] -Arguments: hashpartitioning(ctr_state#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] -(33) HashAggregate [codegen id : 4] +(36) 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] +(37) 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 +(38) 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] +(39) 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] +(40) 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) CometScan parquet spark_catalog.default.customer +(41) CometScan 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 +(42) 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] +(43) CometColumnarToRow [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 +(44) 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] +(45) 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] +(46) 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) CometScan parquet spark_catalog.default.customer_address +(47) CometScan 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 +(48) 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] +(49) CometColumnarToRow [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 +(50) 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] +(51) 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] +(52) 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 +(53) 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 (58) ++- * CometColumnarToRow (57) + +- CometProject (56) + +- CometFilter (55) + +- CometScan parquet spark_catalog.default.date_dim (54) -(51) CometScan parquet spark_catalog.default.date_dim +(54) CometScan 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 +(55) 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 +(56) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(54) ColumnarToRow [codegen id : 1] +(57) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(55) BroadcastExchange +(58) 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 = 20 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 991c00de9f..b2632c7758 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,69 +8,72 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (1) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cr_returning_customer_sk,ca_state] #7 + WholeStageCodegen (2) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + CometColumnarToRow + 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) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow 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] 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 34319a8fc3..6117d4a921 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 42c3f40e68..755128d2c4 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 b7136e1636..9fea80f351 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,57 @@ == 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 (53) ++- * Project (52) + +- * BroadcastHashJoin Inner BuildRight (51) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * HashAggregate (20) + : : +- * CometColumnarToRow (19) + : : +- CometColumnarExchange (18) + : : +- * HashAggregate (17) + : : +- * CometColumnarToRow (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 (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- CometColumnarExchange (31) + : +- * HashAggregate (30) + : +- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (21) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- BroadcastExchange (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- CometColumnarExchange (47) + +- * HashAggregate (46) + +- * CometColumnarToRow (45) + +- CometProject (44) + +- CometBroadcastHashJoin (43) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_returns (37) + : +- ReusedExchange (39) + +- ReusedExchange (42) (1) CometScan parquet spark_catalog.default.store_returns @@ -98,7 +101,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: 63] Output [1]: [d_date#9] (11) CometBroadcastHashJoin @@ -123,7 +126,7 @@ Arguments: [sr_returned_date_sk#3], [d_date_sk#7], Inner, BuildRight Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#7] Arguments: [sr_return_quantity#2, i_item_id#6], [sr_return_quantity#2, i_item_id#6] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [2]: [sr_return_quantity#2, i_item_id#6] (17) HashAggregate [codegen id : 1] @@ -133,18 +136,21 @@ Functions [1]: [partial_sum(sr_return_quantity#2)] Aggregate Attributes [1]: [sum#10] Results [2]: [i_item_id#6, sum#11] -(18) Exchange +(18) CometColumnarExchange Input [2]: [i_item_id#6, sum#11] -Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 6] +(19) CometColumnarToRow [codegen id : 6] +Input [2]: [i_item_id#6, sum#11] + +(20) 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) CometScan parquet spark_catalog.default.catalog_returns +(21) CometScan 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 +158,73 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#17), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct -(21) CometFilter +(22) 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] +(23) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#19, i_item_id#20] -(23) CometBroadcastHashJoin +(24) 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 +(25) 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] +(26) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#21] -(26) CometBroadcastHashJoin +(27) 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 +(28) 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] +(29) CometColumnarToRow [codegen id : 2] Input [2]: [cr_return_quantity#16, i_item_id#20] -(29) HashAggregate [codegen id : 2] +(30) 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 +(31) 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] -(31) HashAggregate [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] +Input [2]: [i_item_id#20, sum#23] + +(33) 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 +(34) 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] +(35) 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] +(36) 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) CometScan parquet spark_catalog.default.web_returns +(37) CometScan 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 +232,162 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#29), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(36) CometFilter +(38) 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] +(39) ReusedExchange [Reuses operator id: 5] Output [2]: [i_item_sk#31, i_item_id#32] -(38) CometBroadcastHashJoin +(40) 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 +(41) 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] +(42) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#33] -(41) CometBroadcastHashJoin +(43) 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 +(44) 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] +(45) CometColumnarToRow [codegen id : 4] Input [2]: [wr_return_quantity#28, i_item_id#32] -(44) HashAggregate [codegen id : 4] +(46) 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 +(47) CometColumnarExchange +Input [2]: [i_item_id#32, sum#35] +Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(48) CometColumnarToRow [codegen id : 5] Input [2]: [i_item_id#32, sum#35] -Arguments: hashpartitioning(i_item_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(46) HashAggregate [codegen id : 5] +(49) 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 +(50) 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] +(51) 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] +(52) Project [codegen id : 6] Output [8]: [item_id#13, sr_item_qty#14, (((cast(sr_item_qty#14 as double) / knownfloatingpointnormalized(normalizenanandzero(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) / knownfloatingpointnormalized(normalizenanandzero(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) / knownfloatingpointnormalized(normalizenanandzero(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 +(53) 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) CometScan parquet spark_catalog.default.date_dim +BroadcastExchange (67) ++- * CometColumnarToRow (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometFilter (55) + : +- CometScan parquet spark_catalog.default.date_dim (54) + +- CometBroadcastExchange (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometScan parquet spark_catalog.default.date_dim (56) + +- CometBroadcastExchange (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) + + +(54) CometScan 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 +(55) CometFilter Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) -(53) CometScan parquet spark_catalog.default.date_dim +(56) CometScan 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) CometScan parquet spark_catalog.default.date_dim +(57) CometScan 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 +(58) 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 +(59) CometProject Input [2]: [d_date#44, d_week_seq#45] Arguments: [d_week_seq#45], [d_week_seq#45] -(57) CometBroadcastExchange +(60) CometBroadcastExchange Input [1]: [d_week_seq#45] Arguments: [d_week_seq#45] -(58) CometBroadcastHashJoin +(61) 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 +(62) CometProject Input [2]: [d_date#9, d_week_seq#43] Arguments: [d_date#9], [d_date#9] -(60) CometBroadcastExchange +(63) CometBroadcastExchange Input [1]: [d_date#9] Arguments: [d_date#9] -(61) CometBroadcastHashJoin +(64) 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 +(65) CometProject Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(63) ColumnarToRow [codegen id : 1] +(66) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(64) BroadcastExchange +(67) 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 = 21 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 = 37 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 8cd3f3f45e..970502017f 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,82 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,sr_return_quantity] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #8 + WholeStageCodegen (2) + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + CometColumnarToRow + 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] - 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_item_id] #10 + WholeStageCodegen (4) + HashAggregate [i_item_id,wr_return_quantity] [sum,sum] + CometColumnarToRow + 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/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 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 535fb70a91..dbd48f9bbe 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,50 @@ == 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 (46) ++- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.web_sales @@ -238,7 +239,7 @@ Arguments: [wr_reason_sk#13], [r_reason_sk#30], Inner, BuildRight Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#30, r_reason_desc#31] Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] -(41) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#31] (42) HashAggregate [codegen id : 1] @@ -248,50 +249,53 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(r_reason_desc#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(44) HashAggregate [codegen id : 2] +(44) CometColumnarToRow [codegen id : 2] +Input [7]: [r_reason_desc#31, sum#38, count#39, sum#40, count#41, sum#42, count#43] + +(45) 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 +(46) 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 (51) ++- * CometColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(46) CometScan parquet spark_catalog.default.date_dim +(47) CometScan 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 +(48) 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 +(49) CometProject Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] -(50) BroadcastExchange +(51) 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 0d350bd591..3972001e3a 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,58 @@ 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 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 [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 018954bf35..14cccc86bc 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,29 @@ == 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 (25) ++- * Project (24) + +- Window (23) + +- * CometColumnarToRow (22) + +- CometSort (21) + +- CometColumnarExchange (20) + +- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometColumnarExchange (17) + +- * HashAggregate (16) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.web_sales @@ -92,7 +94,7 @@ Arguments: [ws_net_paid#2, i_category#9, i_class#8], [ws_net_paid#2, i_category# Input [3]: [ws_net_paid#2, i_category#9, i_class#8] Arguments: [[ws_net_paid#2, i_category#9, i_class#8, 0], [ws_net_paid#2, i_category#9, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] -(15) ColumnarToRow [codegen id : 1] +(15) CometColumnarToRow [codegen id : 1] Input [4]: [ws_net_paid#2, i_category#10, i_class#11, spark_grouping_id#12] (16) HashAggregate [codegen id : 1] @@ -102,66 +104,72 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_category#10, i_class#11, spark_grouping_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(18) HashAggregate [codegen id : 2] +(18) CometColumnarToRow [codegen id : 2] +Input [4]: [i_category#10, i_class#11, spark_grouping_id#12, sum#14] + +(19) 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 +(20) 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] + +(21) 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] +(22) CometColumnarToRow [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 +(23) 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] +(24) 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 +(25) 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 (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(24) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(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)) -(26) CometProject +(28) 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] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(28) BroadcastExchange +(30) 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 e4d028f2fb..720ed320c5 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,38 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (3) - Sort [_w1,_w2,_w0] + CometColumnarToRow 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 + WholeStageCodegen (2) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 28b942e2f7..d496a7ecb2 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,54 @@ == Physical Plan == -* HashAggregate (49) -+- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin LeftAnti BuildRight (45) - :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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 (50) ++- * CometColumnarToRow (49) + +- CometColumnarExchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * CometColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometExchange (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) + : +- * CometColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometExchange (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) + +- * CometColumnarToRow (43) + +- CometHashAggregate (42) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -128,7 +129,7 @@ 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: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +182,7 @@ 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: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +245,7 @@ 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: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -268,11 +269,14 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) CometColumnarExchange Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) HashAggregate [codegen id : 4] +(49) CometColumnarToRow [codegen id : 4] +Input [1]: [count#27] + +(50) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -282,32 +286,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 (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(50) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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 +(53) 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] +(54) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(55) 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 6e3328ae1d..e4f27f3a1b 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,69 +1,70 @@ WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (3) + HashAggregate [count,count] + Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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) + CometColumnarToRow + 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 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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] + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 + CometExchange [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 [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] + 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 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt index 592e23cd27..6682e151f1 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,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange 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 1bb61b6c92..21b72d36a7 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 @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 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 32dc4ffa71..cc6729d1b5 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,34 @@ == 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 (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometColumnarExchange (21) + +- * HashAggregate (20) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.item @@ -117,7 +119,7 @@ Arguments: [ss_store_sk#6], [s_store_sk#13], Inner, BuildRight Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#12, s_store_sk#13, s_store_name#14, s_company_name#15] Arguments: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15], [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#12, s_store_name#14, s_company_name#15] (20) HashAggregate [codegen id : 1] @@ -127,70 +129,76 @@ 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) 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, [plan_id=1] +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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [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] + +(23) 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 +(24) 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] + +(25) 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] +(26) CometColumnarToRow [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 +(27) 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] +(28) 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] +(29) 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 +(30) 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 (35) ++- * CometColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) -(29) CometScan parquet spark_catalog.default.date_dim +(31) CometScan 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 +(32) 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 +(33) 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] +(34) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] -(33) BroadcastExchange +(35) 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 2bfff90d34..e0a8b006a5 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,42 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 eb8fb26dfa..1718c0cdc6 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 @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (4) +* CometColumnarToRow (4) +- CometProject (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -20,20 +20,21 @@ Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) Input [1]: [r_reason_sk#1] Arguments: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6], [CASE WHEN (Subquery scalar-subquery#7, [id=#8].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_net_paid) END AS bucket1#2, CASE WHEN (Subquery scalar-subquery#9, [id=#10].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_net_paid) END AS bucket2#3, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket3#4, CASE WHEN (Subquery scalar-subquery#13, [id=#14].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_net_paid) END AS bucket4#5, CASE WHEN (Subquery scalar-subquery#15, [id=#16].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_net_paid) END AS bucket5#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [5]: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -* Project (12) -+- * HashAggregate (11) - +- Exchange (10) - +- * HashAggregate (9) - +- * ColumnarToRow (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan parquet spark_catalog.default.store_sales (5) +* Project (13) ++- * HashAggregate (12) + +- * CometColumnarToRow (11) + +- CometColumnarExchange (10) + +- * HashAggregate (9) + +- * CometColumnarToRow (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.store_sales (5) (5) CometScan parquet spark_catalog.default.store_sales @@ -51,7 +52,7 @@ Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quant Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] -(8) ColumnarToRow [codegen id : 1] +(8) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] (9) HashAggregate [codegen id : 1] @@ -61,18 +62,21 @@ 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) CometColumnarExchange Input [5]: [count#26, sum#27, count#28, sum#29, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(11) HashAggregate [codegen id : 2] +(11) CometColumnarToRow [codegen id : 2] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] + +(12) 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] +(13) 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 +85,57 @@ Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] Subquery:4 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* Project (20) -+- * HashAggregate (19) - +- Exchange (18) - +- * HashAggregate (17) - +- * ColumnarToRow (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan parquet spark_catalog.default.store_sales (13) - - -(13) CometScan parquet spark_catalog.default.store_sales +* Project (22) ++- * HashAggregate (21) + +- * CometColumnarToRow (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store_sales (14) + + +(14) CometScan 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 +(15) 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 +(16) 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] +(17) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] -(17) HashAggregate [codegen id : 1] +(18) 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 +(19) 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] +(20) CometColumnarToRow [codegen id : 2] +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] + +(21) 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] +(22) 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 +144,57 @@ Subquery:5 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] Subquery:7 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* Project (28) -+- * HashAggregate (27) - +- Exchange (26) - +- * HashAggregate (25) - +- * ColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan parquet spark_catalog.default.store_sales (21) - - -(21) CometScan parquet spark_catalog.default.store_sales +* Project (31) ++- * HashAggregate (30) + +- * CometColumnarToRow (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.store_sales (23) + + +(23) CometScan 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 +(24) 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 +(25) 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] +(26) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] -(25) HashAggregate [codegen id : 1] +(27) 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 +(28) CometColumnarExchange +Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(29) CometColumnarToRow [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] +(30) 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] +(31) 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 +203,57 @@ Subquery:8 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:10 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* Project (36) -+- * HashAggregate (35) - +- Exchange (34) - +- * HashAggregate (33) - +- * ColumnarToRow (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.store_sales (29) - - -(29) CometScan parquet spark_catalog.default.store_sales +* Project (40) ++- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.store_sales (32) + + +(32) CometScan 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 +(33) 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 +(34) 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] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] -(33) HashAggregate [codegen id : 1] +(36) 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 +(37) CometColumnarExchange +Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(38) CometColumnarToRow [codegen id : 2] Input [5]: [count#89, sum#90, count#91, sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] -(35) HashAggregate [codegen id : 2] +(39) 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] +(40) 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 +262,57 @@ Subquery:11 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] Subquery:13 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#15, [id=#16] -* Project (44) -+- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * ColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan parquet spark_catalog.default.store_sales (37) - - -(37) CometScan parquet spark_catalog.default.store_sales +* Project (49) ++- * HashAggregate (48) + +- * CometColumnarToRow (47) + +- CometColumnarExchange (46) + +- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.store_sales (41) + + +(41) CometScan 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 +(42) 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 +(43) 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] +(44) CometColumnarToRow [codegen id : 1] Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] -(41) HashAggregate [codegen id : 1] +(45) 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 +(46) CometColumnarExchange +Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(47) CometColumnarToRow [codegen id : 2] Input [5]: [count#110, sum#111, count#112, sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(43) HashAggregate [codegen id : 2] +(48) 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] +(49) 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 2dfdbe09f5..1b4462fe98 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 @@ -1,80 +1,85 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [bucket1,bucket2,bucket3,bucket4,bucket5] 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + CometColumnarToRow + 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 CometFilter [r_reason_sk] 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 a6ec6f4b96..38379fb61c 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,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange 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 95fd73d86e..72e45a9ad1 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,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 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 a8fa8f8571..41d6616309 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,46 +1,47 @@ == Physical Plan == -* ColumnarToRow (42) -+- CometSort (41) - +- CometColumnarExchange (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) +* CometColumnarToRow (43) ++- CometSort (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.call_center @@ -211,7 +212,7 @@ Arguments: [c_current_hdemo_sk#15], [hd_demo_sk#22], Inner, BuildRight Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#15, cd_marital_status#20, cd_education_status#21, hd_demo_sk#22] Arguments: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21], [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#20, cd_education_status#21] (37) HashAggregate [codegen id : 1] @@ -221,57 +222,60 @@ 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) 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, [plan_id=1] +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] -(39) HashAggregate [codegen id : 2] +(39) CometColumnarToRow [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] + +(40) 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) CometColumnarExchange +(41) 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] -(41) CometSort +(42) CometSort Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] Arguments: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30], [Returns_Loss#30 DESC NULLS LAST] -(42) ColumnarToRow [codegen id : 3] +(43) CometColumnarToRow [codegen id : 3] Input [4]: [Call_Center#27, Call_Center_Name#28, Manager#29, Returns_Loss#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (47) -+- * ColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan parquet spark_catalog.default.date_dim (43) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) -(43) CometScan parquet spark_catalog.default.date_dim +(44) CometScan 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 -(44) CometFilter +(45) 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)) -(45) CometProject +(46) CometProject Input [3]: [d_date_sk#10, d_year#11, d_moy#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(46) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(47) BroadcastExchange +(48) 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 67669c48b7..a48564c34e 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,56 +1,57 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 3981cc6e2f..82e786a6ec 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,37 @@ == 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 (33) ++- * CometColumnarToRow (32) + +- CometColumnarExchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * CometColumnarToRow (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 (24) + : +- * Filter (23) + : +- * HashAggregate (22) + : +- * CometColumnarToRow (21) + : +- CometColumnarExchange (20) + : +- * HashAggregate (19) + : +- * CometColumnarToRow (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 (27) (1) CometScan parquet spark_catalog.default.web_sales @@ -72,7 +74,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] +(9) CometColumnarToRow [codegen id : 4] Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (10) CometScan parquet spark_catalog.default.web_sales @@ -115,7 +117,7 @@ Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] (19) HashAggregate [codegen id : 1] @@ -125,60 +127,66 @@ 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) CometColumnarExchange Input [3]: [ws_item_sk#7, sum#15, count#16] -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] +(21) CometColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#7, sum#15, count#16] + +(22) 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] +(23) 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 +(24) 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] +(25) 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] +(26) 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] +(27) ReusedExchange [Reuses operator id: 38] Output [1]: [d_date_sk#19] -(27) BroadcastHashJoin [codegen id : 4] +(28) 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] +(29) 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] +(30) 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 +(31) CometColumnarExchange +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(32) CometColumnarToRow [codegen id : 5] Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(31) HashAggregate [codegen id : 5] +(33) HashAggregate [codegen id : 5] Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] @@ -188,32 +196,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 (38) ++- * CometColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) -(32) CometScan parquet spark_catalog.default.date_dim +(34) CometScan 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 +(35) 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 +(36) CometProject Input [2]: [d_date_sk#19, d_date#24] Arguments: [d_date_sk#19], [d_date_sk#19] -(35) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] -(36) BroadcastExchange +(38) 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 aa24a049b2..9612eff307 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,52 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #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))] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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) - 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ws_item_sk] #5 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + CometColumnarToRow + 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 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 d6e9142db4..04f9777b85 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,26 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (22) -+- * HashAggregate (21) - +- Exchange (20) - +- * HashAggregate (19) - +- * ColumnarToRow (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (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) +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- * CometColumnarToRow (21) + +- CometColumnarExchange (20) + +- * HashAggregate (19) + +- * CometColumnarToRow (18) + +- CometProject (17) + +- CometBroadcastHashJoin (16) + :- CometProject (11) + : +- CometSortMergeJoin (10) + : :- CometSort (4) + : : +- CometExchange (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- CometSort (9) + : +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -101,7 +102,7 @@ Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight 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] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [2]: [ss_customer_sk#2, act_sales#14] (19) HashAggregate [codegen id : 1] @@ -111,18 +112,21 @@ Functions [1]: [partial_sum(act_sales#14)] Aggregate Attributes [2]: [sum#15, isEmpty#16] Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -(20) Exchange +(20) CometColumnarExchange Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [codegen id : 2] +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] + +(22) 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)] Aggregate Attributes [1]: [sum(act_sales#14)#19] Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] -(22) TakeOrderedAndProject +(23) 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] 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 fb8e4602dd..44702cd3de 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,26 +1,27 @@ TakeOrderedAndProject [sumsales,ss_customer_sk] WholeStageCodegen (2) HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] - InputAdapter - Exchange [ss_customer_sk] #1 - 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] - CometExchange [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] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ss_customer_sk] #1 + WholeStageCodegen (1) + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 70aaeb9b6e..bad20fa5a2 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * HashAggregate (36) - +- * ColumnarToRow (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.web_site (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.web_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.web_site (29) (1) CometScan parquet spark_catalog.default.web_sales @@ -198,7 +199,7 @@ Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] (36) HashAggregate [codegen id : 1] @@ -222,11 +223,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(Unscaled 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] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#22, sum#23, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#22, sum#23, count#25] + +(41) HashAggregate [codegen id : 2] 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 4d0f1f982b..db01381605 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - 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] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [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_order_number,ws_warehouse_sk] + 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] + CometExchange [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] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [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] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 4963b31940..1f4ee1876a 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,57 +1,58 @@ == Physical Plan == -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * ColumnarToRow (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.web_site (42) +* HashAggregate (54) ++- * CometColumnarToRow (53) + +- CometColumnarExchange (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- CometProject (47) + +- CometBroadcastHashJoin (46) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometSortMergeJoin (29) + : : : :- CometSortMergeJoin (15) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometProject (14) + : : : : +- CometSortMergeJoin (13) + : : : : :- CometSort (10) + : : : : : +- CometExchange (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : : +- CometSort (12) + : : : : +- ReusedExchange (11) + : : : +- CometProject (28) + : : : +- CometSortMergeJoin (27) + : : : :- CometSort (20) + : : : : +- CometExchange (19) + : : : : +- CometProject (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.web_returns (16) + : : : +- CometProject (26) + : : : +- CometSortMergeJoin (25) + : : : :- CometSort (22) + : : : : +- ReusedExchange (21) + : : : +- CometSort (24) + : : : +- ReusedExchange (23) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.date_dim (30) + : +- CometBroadcastExchange (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer_address (36) + +- CometBroadcastExchange (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.web_site (42) (1) CometScan parquet spark_catalog.default.web_sales @@ -265,7 +266,7 @@ Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] (49) HashAggregate [codegen id : 1] @@ -289,11 +290,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(Unscaled 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] -(52) Exchange +(52) CometColumnarExchange Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) HashAggregate [codegen id : 2] +(53) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#27, sum#28, count#30] + +(54) HashAggregate [codegen id : 2] 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 b6d5c5c365..fa782fb8e4 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,57 +1,58 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [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] CometProject [ws_order_number] CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometExchange [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] CometSort [ws_warehouse_sk,ws_order_number] ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - 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 [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [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] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 d2e63bee25..677abdadae 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 d9a87aa3c4..9e86a025ee 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 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 1be5c9b947..c09b602f5f 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,29 +1,30 @@ == Physical Plan == -* HashAggregate (25) -+- Exchange (24) - +- * HashAggregate (23) - +- * ColumnarToRow (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (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) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) +* HashAggregate (26) ++- * CometColumnarToRow (25) + +- CometColumnarExchange (24) + +- * HashAggregate (23) + +- * CometColumnarToRow (22) + +- CometProject (21) + +- CometSortMergeJoin (20) + :- CometSort (11) + : +- CometHashAggregate (10) + : +- CometExchange (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) + +- CometExchange (17) + +- CometHashAggregate (16) + +- CometProject (15) + +- CometBroadcastHashJoin (14) + :- CometScan parquet spark_catalog.default.catalog_sales (12) + +- ReusedExchange (13) (1) CometScan parquet spark_catalog.default.store_sales @@ -125,7 +126,7 @@ Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter 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] +(22) CometColumnarToRow [codegen id : 1] Input [2]: [customer_sk#7, customer_sk#14] (23) HashAggregate [codegen id : 1] @@ -135,11 +136,14 @@ Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(custo Aggregate Attributes [3]: [sum#16, sum#17, sum#18] Results [3]: [sum#19, sum#20, sum#21] -(24) Exchange +(24) CometColumnarExchange Input [3]: [sum#19, sum#20, sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(25) HashAggregate [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#19, sum#20, sum#21] + +(26) 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)] @@ -149,32 +153,32 @@ 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 (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(26) CometScan parquet spark_catalog.default.date_dim +(27) CometScan 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 -(27) 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)) -(28) CometProject +(29) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#5] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt index a9a3e66eb6..9ac127dc0c 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,38 +1,39 @@ 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 (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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 b750a8833e..114c14db5d 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,29 +1,31 @@ == Physical Plan == -* ColumnarToRow (25) -+- CometProject (24) - +- CometSort (23) - +- CometColumnarExchange (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) +* CometColumnarToRow (27) ++- CometProject (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -90,7 +92,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -100,77 +102,83 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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) CometColumnarExchange +(24) 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: 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] -(23) CometSort +(25) 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] -(24) CometProject +(26) 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] -(25) ColumnarToRow [codegen id : 5] +(27) CometColumnarToRow [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 (30) -+- * ColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan parquet spark_catalog.default.date_dim (26) +BroadcastExchange (32) ++- * CometColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) -(26) CometScan parquet spark_catalog.default.date_dim +(28) CometScan 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 -(27) CometFilter +(29) 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)) -(28) CometProject +(30) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(30) BroadcastExchange +(32) 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 b2c6adb5bb..443e79ccd9 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow 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] @@ -9,35 +9,37 @@ WholeStageCodegen (5) InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 [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_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 f94f48c207..9114546cf5 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,33 @@ == 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 (29) ++- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- CometColumnarExchange (26) + +- * HashAggregate (25) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -140,7 +141,7 @@ Arguments: [cs_ship_date_sk#1], [d_date_sk#12], Inner, BuildRight Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11, d_date_sk#12] Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] -(24) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] (25) HashAggregate [codegen id : 1] @@ -150,18 +151,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(27) HashAggregate [codegen id : 2] +(27) CometColumnarToRow [codegen id : 2] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(28) 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 +(29) 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 25ff3a2fed..c9de30b7d8 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,33 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [_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] + CometColumnarToRow + 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 27813d7e70..f9a474c65f 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometTakeOrderedAndProject (39) +- CometProject (38) +- CometBroadcastHashJoin (37) @@ -224,14 +224,14 @@ Arguments: [c_customer_id#21], [c_customer_id#21] 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] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [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 (45) -+- * ColumnarToRow (44) ++- * CometColumnarToRow (44) +- CometProject (43) +- CometFilter (42) +- CometScan parquet spark_catalog.default.date_dim (41) @@ -252,7 +252,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (45) BroadcastExchange 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 10d410761e..707c2ff0aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_customer_id] CometProject [c_customer_id] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 e32e86f270..3c2df96933 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,50 @@ == 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 (46) ++- * HashAggregate (45) + +- * CometColumnarToRow (44) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (33) + : +- * CometColumnarToRow (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (39) + +- * CometColumnarToRow (38) + +- CometFilter (37) + +- CometScan parquet spark_catalog.default.customer_demographics (36) (1) CometScan parquet spark_catalog.default.customer @@ -101,7 +102,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -123,7 +124,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -155,7 +156,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -191,7 +192,7 @@ Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana Input [2]: [ca_address_sk#20, ca_county#21] Arguments: [ca_address_sk#20], [ca_address_sk#20] -(32) ColumnarToRow [codegen id : 3] +(32) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#20] (33) BroadcastExchange @@ -219,7 +220,7 @@ ReadSchema: struct -(47) CometFilter +(48) 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 +(49) 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] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(50) 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/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt index efd4b187dd..bc0ed49101 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,72 @@ 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + 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) + CometColumnarToRow 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 #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 609ab39c86..a01c703239 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (69) +* CometColumnarToRow (69) +- CometTakeOrderedAndProject (68) +- CometProject (67) +- CometBroadcastHashJoin (66) @@ -393,14 +393,14 @@ Arguments: [customer_preferred_cust_flag#36], [customer_preferred_cust_flag#36] 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] +(69) CometColumnarToRow [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 (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -416,7 +416,7 @@ ReadSchema: struct 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)) -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (73) BroadcastExchange @@ -425,7 +425,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometFilter (75) +- CometScan parquet spark_catalog.default.date_dim (74) @@ -441,7 +441,7 @@ ReadSchema: struct 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)) -(76) ColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (77) BroadcastExchange 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 e509c8c3c1..8896bc14be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_preferred_cust_flag] CometProject [customer_preferred_cust_flag] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -47,7 +47,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 18abad247b..7b8864d54f 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 e81a507139..c4c49fe339 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 e3c357b407..774d652f36 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometHashAggregate (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -188,14 +188,14 @@ 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))] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 BroadcastExchange (38) -+- * ColumnarToRow (37) ++- * CometColumnarToRow (37) +- CometProject (36) +- CometFilter (35) +- CometScan parquet spark_catalog.default.date_dim (34) @@ -216,7 +216,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (38) BroadcastExchange 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 46ed9b6365..c4d3b744d0 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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))] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 3aae3a13ec..868da4cd92 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometTakeOrderedAndProject (101) +- CometHashAggregate (100) +- CometExchange (99) @@ -569,13 +569,13 @@ Functions [2]: [sum(sales#49), sum(number_sales#50)] 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] +(102) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* ColumnarToRow (119) +* CometColumnarToRow (119) +- CometHashAggregate (118) +- CometExchange (117) +- CometHashAggregate (116) @@ -670,7 +670,7 @@ Input [2]: [sum#120, count#121] Keys: [] Functions [1]: [avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#122] Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#12 @@ -681,7 +681,7 @@ Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (124) -+- * ColumnarToRow (123) ++- * CometColumnarToRow (123) +- CometProject (122) +- CometFilter (121) +- CometScan parquet spark_catalog.default.date_dim (120) @@ -702,7 +702,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(123) ColumnarToRow [codegen id : 1] +(123) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (124) BroadcastExchange @@ -711,7 +711,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (129) -+- * ColumnarToRow (128) ++- * CometColumnarToRow (128) +- CometProject (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -732,7 +732,7 @@ Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 < Input [2]: [d_date_sk#26, d_year#123] Arguments: [d_date_sk#26], [d_date_sk#26] -(128) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (129) BroadcastExchange 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 852af8ad82..ec0b46d611 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -11,7 +11,7 @@ WholeStageCodegen (1) CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 @@ -45,7 +45,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -69,7 +69,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #7 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 47feaf8987..a3ed5f9e1c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_c 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] +(85) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#97] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1999)) AND (d_year#102 < Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#103] 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 bb4be4d3ae..befc6dadbb 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,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] 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 037c526267..3ee326ad53 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (23) +* CometColumnarToRow (23) +- CometTakeOrderedAndProject (22) +- CometHashAggregate (21) +- CometExchange (20) @@ -130,14 +130,14 @@ Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] 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] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ca_zip#9, sum(cs_sales_price)#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -158,7 +158,7 @@ Condition : ((((isnotnull(d_qoy#12) AND isnotnull(d_year#11)) AND (d_qoy#12 = 2) 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] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (28) BroadcastExchange 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 d417f1c682..425f20e10c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 acd12b2771..dc963d61f6 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * ColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.call_center (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * CometColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.call_center (29) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -203,7 +204,7 @@ 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))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [cs_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -220,11 +221,14 @@ Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] Results [3]: [sum#20, sum#21, count#25] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#20, sum#21, count#25] + +(41) HashAggregate [codegen id : 2] Input [3]: [sum#20, sum#21, 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 f054ee03e9..daecc86311 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - 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] - CometExchange [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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [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] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_order_number,sum,sum,cs_ext_ship_cost,cs_net_profit] + CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] + CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] + CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] + CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] + CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + CometSortMergeJoin [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_order_number,cs_warehouse_sk] + 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] + CometExchange [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] + CometSort [cs_warehouse_sk,cs_order_number] + CometExchange [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] + CometSort [cr_order_number] + CometExchange [cr_order_number] #4 + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [cc_call_center_sk] #7 + 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 496ec2f1a1..08c00a0bc4 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_qu 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] +(42) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((isnotnull(d_quarter_name#20) AND (d_quarter_name#20 = 2001Q1)) AND Input [2]: [d_date_sk#19, d_quarter_name#20] Arguments: [d_date_sk#19], [d_date_sk#19] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (d_quarter_name#22 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk Input [2]: [d_date_sk#21, d_quarter_name#22] Arguments: [d_date_sk#21], [d_date_sk#21] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#21] (52) BroadcastExchange 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 a3adaf1724..97000b91df 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_quarter_name] 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 c41a1ef7d4..2d08a0303c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometTakeOrderedAndProject (40) +- CometHashAggregate (39) +- CometExchange (38) @@ -232,14 +232,14 @@ Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_pric 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] +(41) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(45) ColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (46) BroadcastExchange 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 e26491c76a..eaf02960fe 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)))] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 b91551e576..c1bf0f14a0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -193,6 +193,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] 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] +(34) CometColumnarToRow [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 46b0d650a5..bfbf7fe94a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 24c39591f5..0baaf2cdfa 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (33) +* CometColumnarToRow (33) +- CometSort (32) +- CometColumnarExchange (31) +- CometProject (30) @@ -183,6 +183,6 @@ Arguments: rangepartitioning(d_week_seq1#28 ASC NULLS FIRST, 5), ENSURE_REQUIREM 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] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [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 ed772bade0..38cb758b73 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 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 909acbf7f8..6a32155103 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 9a906085b8..147d91b586 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 31a5bef868..0340ef91c8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometFilter (22) +- CometHashAggregate (21) @@ -135,14 +135,14 @@ Condition : (CASE WHEN (inv_before#15 > 0) THEN (knownfloatingpointnormalized(no 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) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [codegen id : 1] Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#15, inv_after#16] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ ReadSchema: struct 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] +(27) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#11, d_date#12] (28) BroadcastExchange 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 6e483f6049..6ff97b53ac 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 526dadb3c8..759689b1b8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometTakeOrderedAndProject (23) +- CometHashAggregate (22) +- CometExchange (21) @@ -135,14 +135,14 @@ Functions [1]: [avg(inv_quantity_on_hand#3)] 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] +(24) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (29) BroadcastExchange 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 ed997b5d50..f9d4ce909d 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 45dd2f9be2..9c0af9c56d 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (67) +* CometColumnarToRow (67) +- CometHashAggregate (66) +- CometExchange (65) +- CometHashAggregate (64) @@ -370,14 +370,14 @@ Input [2]: [sum#41, isEmpty#42] Keys: [] Functions [1]: [sum(sales#32)] -(67) ColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [sum(sales)#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometProject (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -398,7 +398,7 @@ Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2 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] +(71) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#29] (72) BroadcastExchange @@ -407,7 +407,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (77) -+- * ColumnarToRow (76) ++- * CometColumnarToRow (76) +- CometProject (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -428,7 +428,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) 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] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (77) BroadcastExchange @@ -436,7 +436,7 @@ Input [2]: [d_date_sk#10, d_date#11] 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) +* CometColumnarToRow (95) +- CometHashAggregate (94) +- CometExchange (93) +- CometHashAggregate (92) @@ -536,12 +536,12 @@ Input [1]: [max#55] Keys: [] Functions [1]: [max(csales#54)] -(95) ColumnarToRow [codegen id : 1] +(95) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#56] Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 BroadcastExchange (100) -+- * ColumnarToRow (99) ++- * CometColumnarToRow (99) +- CometProject (98) +- CometFilter (97) +- CometScan parquet spark_catalog.default.date_dim (96) @@ -562,7 +562,7 @@ Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) Input [2]: [d_date_sk#50, d_year#51] Arguments: [d_date_sk#50], [d_date_sk#50] -(99) ColumnarToRow [codegen id : 1] +(99) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#50] (100) BroadcastExchange 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 7606f105f3..4ecd0dfcc5 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -37,7 +37,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -54,7 +54,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 4a4b3a41ca..056bcc611d 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (87) +* CometColumnarToRow (87) +- CometTakeOrderedAndProject (86) +- CometUnion (85) :- CometHashAggregate (62) @@ -479,14 +479,14 @@ Child 1 Input [3]: [c_last_name#45, c_first_name#44, sales#50] 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] +(87) CometColumnarToRow [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 (92) -+- * ColumnarToRow (91) ++- * CometColumnarToRow (91) +- CometProject (90) +- CometFilter (89) +- CometScan parquet spark_catalog.default.date_dim (88) @@ -507,7 +507,7 @@ Condition : ((((isnotnull(d_year#33) AND isnotnull(d_moy#34)) AND (d_year#33 = 2 Input [3]: [d_date_sk#32, d_year#33, d_moy#34] Arguments: [d_date_sk#32], [d_date_sk#32] -(91) ColumnarToRow [codegen id : 1] +(91) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#32] (92) BroadcastExchange @@ -516,7 +516,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (97) -+- * ColumnarToRow (96) ++- * CometColumnarToRow (96) +- CometProject (95) +- CometFilter (94) +- CometScan parquet spark_catalog.default.date_dim (93) @@ -537,7 +537,7 @@ Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) Input [3]: [d_date_sk#10, d_date#11, d_year#12] Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] -(96) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_date#11] (97) BroadcastExchange @@ -545,7 +545,7 @@ Input [2]: [d_date_sk#10, d_date#11] 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) +* CometColumnarToRow (115) +- CometHashAggregate (114) +- CometExchange (113) +- CometHashAggregate (112) @@ -645,12 +645,12 @@ Input [1]: [max#62] Keys: [] Functions [1]: [max(csales#61)] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [tpcds_cmax#63] Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 BroadcastExchange (120) -+- * ColumnarToRow (119) ++- * CometColumnarToRow (119) +- CometProject (118) +- CometFilter (117) +- CometScan parquet spark_catalog.default.date_dim (116) @@ -671,7 +671,7 @@ Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) Input [2]: [d_date_sk#57, d_year#58] Arguments: [d_date_sk#57], [d_date_sk#57] -(119) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#57] (120) BroadcastExchange 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 d503bb68af..9493bffd25 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -40,7 +40,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) CometFilter [c_customer_sk,ssales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [tpcds_cmax,max,max(csales)] CometExchange #11 @@ -74,7 +74,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #13 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 2208ce2663..6605511424 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,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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 -(53) CometFilter +(55) 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) -(54) CometBroadcastExchange +(56) 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] -(55) CometBroadcastHashJoin +(57) 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 -(56) CometProject +(58) 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] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(58) CometBroadcastHashJoin +(60) 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 -(59) CometProject +(61) 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] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [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] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(62) BroadcastHashJoin [codegen id : 2] +(64) 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 -(63) Project [codegen id : 2] +(65) 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] -(64) HashAggregate [codegen id : 2] +(66) 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#40] 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#41] -(65) Exchange +(67) 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#41] +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=6] + +(68) CometColumnarToRow [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#41] -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=6] -(66) HashAggregate [codegen id : 3] +(69) 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#41] 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 [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#31] Keys: [] Functions [1]: [partial_avg(netpaid#31)] Aggregate Attributes [2]: [sum#42, count#43] Results [2]: [sum#44, count#45] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#44, count#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#44, count#45] Keys: [] Functions [1]: [avg(netpaid#31)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt index c3b9566431..5a6eeed751 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt @@ -1,46 +1,48 @@ == Physical Plan == -* Filter (42) -+- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* Filter (44) ++- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -174,7 +176,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] -(29) ColumnarToRow [codegen id : 2] +(29) CometColumnarToRow [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) CometScan parquet spark_catalog.default.customer_address @@ -188,7 +190,7 @@ ReadSchema: struct Input [3]: [ca_state#25, ca_zip#26, ca_country#27] Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] (33) BroadcastExchange @@ -212,188 +214,202 @@ 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] -(37) Exchange +(37) 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, [plan_id=4] +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] -(38) HashAggregate [codegen id : 3] +(38) CometColumnarToRow [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] + +(39) 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] -(39) HashAggregate [codegen id : 3] +(40) 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] -(40) Exchange +(41) 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] + +(42) CometColumnarToRow [codegen id : 4] 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=5] -(41) HashAggregate [codegen id : 4] +(43) 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] -(42) Filter [codegen id : 4] +(44) 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 = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (69) -+- Exchange (68) - +- * HashAggregate (67) - +- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- * Project (63) - +- * BroadcastHashJoin Inner BuildRight (62) - :- * ColumnarToRow (60) - : +- CometProject (59) - : +- CometBroadcastHashJoin (58) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometProject (48) - : : : : +- CometSortMergeJoin (47) - : : : : :- CometSort (44) - : : : : : +- ReusedExchange (43) - : : : : +- CometSort (46) - : : : : +- ReusedExchange (45) - : : : +- ReusedExchange (49) - : : +- CometBroadcastExchange (54) - : : +- CometFilter (53) - : : +- CometScan parquet spark_catalog.default.item (52) - : +- ReusedExchange (57) - +- ReusedExchange (61) - - -(43) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- * CometColumnarToRow (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometColumnarExchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * CometColumnarToRow (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (58) + : : +- CometBroadcastHashJoin (57) + : : :- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometProject (50) + : : : : +- CometSortMergeJoin (49) + : : : : :- CometSort (46) + : : : : : +- ReusedExchange (45) + : : : : +- CometSort (48) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (51) + : : +- CometBroadcastExchange (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.item (54) + : +- ReusedExchange (59) + +- ReusedExchange (63) + + +(45) 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] -(44) CometSort +(46) 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_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] -(45) ReusedExchange [Reuses operator id: 9] +(47) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(46) CometSort +(48) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(47) CometSortMergeJoin +(49) 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 -(48) CometProject +(50) 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] -(49) ReusedExchange [Reuses operator id: 16] +(51) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(50) CometBroadcastHashJoin +(52) 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 -(51) CometProject +(53) 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] -(52) CometScan parquet spark_catalog.default.item +(54) CometScan 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 -(53) CometFilter +(55) 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) -(54) CometBroadcastExchange +(56) 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] -(55) CometBroadcastHashJoin +(57) 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 -(56) CometProject +(58) 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] -(57) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] -(58) CometBroadcastHashJoin +(60) 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 -(59) CometProject +(61) 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] -(60) ColumnarToRow [codegen id : 2] +(62) CometColumnarToRow [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] -(61) ReusedExchange [Reuses operator id: 33] +(63) ReusedExchange [Reuses operator id: 33] Output [3]: [ca_state#25, ca_zip#26, ca_country#27] -(62) BroadcastHashJoin [codegen id : 2] +(64) 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 -(63) Project [codegen id : 2] +(65) 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] -(64) HashAggregate [codegen id : 2] +(66) 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#40] 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#41] -(65) Exchange +(67) 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#41] +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=6] + +(68) CometColumnarToRow [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#41] -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=6] -(66) HashAggregate [codegen id : 3] +(69) 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#41] 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 [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] -(67) HashAggregate [codegen id : 3] +(70) HashAggregate [codegen id : 3] Input [1]: [netpaid#31] Keys: [] Functions [1]: [partial_avg(netpaid#31)] Aggregate Attributes [2]: [sum#42, count#43] Results [2]: [sum#44, count#45] -(68) Exchange +(71) CometColumnarExchange +Input [2]: [sum#44, count#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(72) CometColumnarToRow [codegen id : 4] Input [2]: [sum#44, count#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(69) HashAggregate [codegen id : 4] +(73) HashAggregate [codegen id : 4] Input [2]: [sum#44, count#45] Keys: [] Functions [1]: [avg(netpaid#31)] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt index bd14d82504..df6657f731 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt @@ -3,18 +3,55 @@ WholeStageCodegen (4) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #9 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + 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] + CometColumnarToRow + 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] #10 + 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_birth_country,s_zip,ca_country,ca_zip] + CometColumnarToRow + 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_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_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] #3 + CometSort [sr_item_sk,sr_ticket_number] + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + 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_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 + 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] + CometColumnarToRow 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] #10 + 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] #2 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_birth_country,s_zip,ca_country,ca_zip] - ColumnarToRow + CometColumnarToRow 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_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_first_name,c_last_name,c_birth_country] @@ -25,62 +62,29 @@ WholeStageCodegen (4) 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] #3 + CometExchange [ss_ticket_number,ss_item_sk] #3 + 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] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 + CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 + 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] #6 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_first_name,c_last_name,c_birth_country] #7 + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - 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] #1 - 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] - 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] #2 - 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_birth_country,s_zip,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_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_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] - CometExchange [ss_ticket_number,ss_item_sk] #3 - 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] - CometExchange [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 [s_store_sk,s_store_name,s_state,s_zip] #5 - 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] #6 - 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_first_name,c_last_name,c_birth_country] #7 - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometFilter [ca_state,ca_zip,ca_country] - CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [ca_state,ca_zip,ca_country] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 6c78335481..62c54c9f56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -239,14 +239,14 @@ Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_lo Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#31 ASC NULLS FIRST,s_store_id#27 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#31,s_store_id#27,s_store_name#28,store_sales_profit#35,store_returns_loss#36,catalog_sales_profit#37]), [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37], 100, [i_item_id#30 ASC NULLS FIRST, i_item_desc#31 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#30, i_item_desc#31, s_store_id#27, s_store_name#28, store_sales_profit#35, store_returns_loss#36, catalog_sales_profit#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -267,7 +267,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4) 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] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (47) BroadcastExchange @@ -276,7 +276,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -297,7 +297,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 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] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (52) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index 13cb05053c..e64e8bc840 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index b93df3398e..525313e3f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(Uns 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] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt index f643ff7b04..16f8d0848e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 4d1b12626a..28bc38292b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#20 ASC NULLS FIRST,s_state#21 ASC NULLS FIRST], output=[i_item_id#20,s_state#21,g_state#31,agg1#32,agg2#33,agg3#34,agg4#35]), [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35], 100, [i_item_id#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST], [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#20, s_state#21, g_state#31, agg1#32, agg2#33, agg3#34, agg4#35] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt index 121f1be7ce..e65b02d4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,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))] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 8f0671c8f0..98f8e50c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -1,74 +1,80 @@ == 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) -: : : : : +- * ColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (21) -: : : : +- * HashAggregate (20) -: : : : +- Exchange (19) -: : : : +- * HashAggregate (18) -: : : : +- * HashAggregate (17) -: : : : +- * ColumnarToRow (16) -: : : : +- CometExchange (15) -: : : : +- CometHashAggregate (14) -: : : : +- CometProject (13) -: : : : +- CometFilter (12) -: : : : +- CometScan parquet spark_catalog.default.store_sales (11) -: : : +- BroadcastExchange (33) -: : : +- * HashAggregate (32) -: : : +- Exchange (31) -: : : +- * HashAggregate (30) -: : : +- * HashAggregate (29) -: : : +- * ColumnarToRow (28) -: : : +- CometExchange (27) -: : : +- CometHashAggregate (26) -: : : +- CometProject (25) -: : : +- CometFilter (24) -: : : +- CometScan parquet spark_catalog.default.store_sales (23) -: : +- BroadcastExchange (45) -: : +- * HashAggregate (44) -: : +- Exchange (43) -: : +- * HashAggregate (42) -: : +- * HashAggregate (41) -: : +- * ColumnarToRow (40) -: : +- CometExchange (39) -: : +- CometHashAggregate (38) -: : +- CometProject (37) -: : +- CometFilter (36) -: : +- CometScan parquet spark_catalog.default.store_sales (35) -: +- BroadcastExchange (57) -: +- * HashAggregate (56) -: +- Exchange (55) -: +- * HashAggregate (54) -: +- * HashAggregate (53) -: +- * ColumnarToRow (52) -: +- CometExchange (51) -: +- CometHashAggregate (50) -: +- CometProject (49) -: +- CometFilter (48) -: +- CometScan parquet spark_catalog.default.store_sales (47) -+- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * HashAggregate (65) - +- * ColumnarToRow (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan parquet spark_catalog.default.store_sales (59) +* BroadcastNestedLoopJoin Inner BuildRight (76) +:- * BroadcastNestedLoopJoin Inner BuildRight (63) +: :- * BroadcastNestedLoopJoin Inner BuildRight (50) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) +: : : : :- * HashAggregate (11) +: : : : : +- * CometColumnarToRow (10) +: : : : : +- CometColumnarExchange (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * HashAggregate (7) +: : : : : +- * CometColumnarToRow (6) +: : : : : +- CometExchange (5) +: : : : : +- CometHashAggregate (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (23) +: : : : +- * HashAggregate (22) +: : : : +- * CometColumnarToRow (21) +: : : : +- CometColumnarExchange (20) +: : : : +- * HashAggregate (19) +: : : : +- * HashAggregate (18) +: : : : +- * CometColumnarToRow (17) +: : : : +- CometExchange (16) +: : : : +- CometHashAggregate (15) +: : : : +- CometProject (14) +: : : : +- CometFilter (13) +: : : : +- CometScan parquet spark_catalog.default.store_sales (12) +: : : +- BroadcastExchange (36) +: : : +- * HashAggregate (35) +: : : +- * CometColumnarToRow (34) +: : : +- CometColumnarExchange (33) +: : : +- * HashAggregate (32) +: : : +- * HashAggregate (31) +: : : +- * CometColumnarToRow (30) +: : : +- CometExchange (29) +: : : +- CometHashAggregate (28) +: : : +- CometProject (27) +: : : +- CometFilter (26) +: : : +- CometScan parquet spark_catalog.default.store_sales (25) +: : +- BroadcastExchange (49) +: : +- * HashAggregate (48) +: : +- * CometColumnarToRow (47) +: : +- CometColumnarExchange (46) +: : +- * HashAggregate (45) +: : +- * HashAggregate (44) +: : +- * CometColumnarToRow (43) +: : +- CometExchange (42) +: : +- CometHashAggregate (41) +: : +- CometProject (40) +: : +- CometFilter (39) +: : +- CometScan parquet spark_catalog.default.store_sales (38) +: +- BroadcastExchange (62) +: +- * HashAggregate (61) +: +- * CometColumnarToRow (60) +: +- CometColumnarExchange (59) +: +- * HashAggregate (58) +: +- * HashAggregate (57) +: +- * CometColumnarToRow (56) +: +- CometExchange (55) +: +- CometHashAggregate (54) +: +- CometProject (53) +: +- CometFilter (52) +: +- CometScan parquet spark_catalog.default.store_sales (51) ++- BroadcastExchange (75) + +- * HashAggregate (74) + +- * CometColumnarToRow (73) + +- CometColumnarExchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.store_sales (64) (1) CometScan parquet spark_catalog.default.store_sales @@ -95,7 +101,7 @@ Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_li Input [4]: [ss_list_price#3, sum#6, count#7, count#8] Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(6) ColumnarToRow [codegen id : 1] +(6) CometColumnarToRow [codegen id : 1] Input [4]: [ss_list_price#3, sum#6, count#7, count#8] (7) HashAggregate [codegen id : 1] @@ -112,314 +118,332 @@ Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_p Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [4]: [sum#6, count#7, count#8, count#12] -(9) Exchange +(9) CometColumnarExchange Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(10) HashAggregate [codegen id : 12] +(10) CometColumnarToRow [codegen id : 12] +Input [4]: [sum#6, count#7, count#8, count#12] + +(11) HashAggregate [codegen id : 12] Input [4]: [sum#6, count#7, count#8, 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))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#9 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#10 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] -(11) CometScan parquet spark_catalog.default.store_sales +(12) CometScan 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 +(13) 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 +(14) 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 +(15) 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) CometExchange +(16) CometExchange Input [4]: [ss_list_price#18, sum#21, count#22, count#23] Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(16) ColumnarToRow [codegen id : 2] +(17) CometColumnarToRow [codegen id : 2] Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -(17) HashAggregate [codegen id : 2] +(18) 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 : 2] +(19) 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 +(20) CometColumnarExchange Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(20) HashAggregate [codegen id : 3] +(21) CometColumnarToRow [codegen id : 3] +Input [4]: [sum#21, count#22, count#23, count#27] + +(22) HashAggregate [codegen id : 3] 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 +(23) BroadcastExchange Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] Arguments: IdentityBroadcastMode, [plan_id=5] -(22) BroadcastNestedLoopJoin [codegen id : 12] +(24) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(23) CometScan parquet spark_catalog.default.store_sales +(25) CometScan 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 +(26) 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 +(27) 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 +(28) 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) CometExchange +(29) CometExchange Input [4]: [ss_list_price#33, sum#36, count#37, count#38] Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(28) ColumnarToRow [codegen id : 4] +(30) CometColumnarToRow [codegen id : 4] Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -(29) HashAggregate [codegen id : 4] +(31) 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 : 4] +(32) 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 +(33) CometColumnarExchange +Input [4]: [sum#36, count#37, count#38, count#42] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(34) CometColumnarToRow [codegen id : 5] Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] -(32) HashAggregate [codegen id : 5] +(35) HashAggregate [codegen id : 5] 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 +(36) BroadcastExchange Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] Arguments: IdentityBroadcastMode, [plan_id=8] -(34) BroadcastNestedLoopJoin [codegen id : 12] +(37) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(35) CometScan parquet spark_catalog.default.store_sales +(38) CometScan 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 +(39) 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 +(40) 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 +(41) 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) CometExchange +(42) CometExchange Input [4]: [ss_list_price#48, sum#51, count#52, count#53] Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] -(40) ColumnarToRow [codegen id : 6] +(43) CometColumnarToRow [codegen id : 6] Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -(41) HashAggregate [codegen id : 6] +(44) 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 : 6] +(45) 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 +(46) CometColumnarExchange +Input [4]: [sum#51, count#52, count#53, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(47) CometColumnarToRow [codegen id : 7] Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(44) HashAggregate [codegen id : 7] +(48) HashAggregate [codegen id : 7] 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 +(49) BroadcastExchange Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] Arguments: IdentityBroadcastMode, [plan_id=11] -(46) BroadcastNestedLoopJoin [codegen id : 12] +(50) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(47) CometScan parquet spark_catalog.default.store_sales +(51) CometScan 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 +(52) 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 +(53) 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 +(54) 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) CometExchange +(55) CometExchange Input [4]: [ss_list_price#63, sum#66, count#67, count#68] Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(52) ColumnarToRow [codegen id : 8] +(56) CometColumnarToRow [codegen id : 8] Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -(53) HashAggregate [codegen id : 8] +(57) 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 : 8] +(58) 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 +(59) CometColumnarExchange Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(56) HashAggregate [codegen id : 9] +(60) CometColumnarToRow [codegen id : 9] +Input [4]: [sum#66, count#67, count#68, count#72] + +(61) HashAggregate [codegen id : 9] 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 +(62) BroadcastExchange Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] Arguments: IdentityBroadcastMode, [plan_id=14] -(58) BroadcastNestedLoopJoin [codegen id : 12] +(63) BroadcastNestedLoopJoin [codegen id : 12] Join type: Inner Join condition: None -(59) CometScan parquet spark_catalog.default.store_sales +(64) CometScan 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 +(65) 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 +(66) 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 +(67) 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) CometExchange +(68) CometExchange Input [4]: [ss_list_price#78, sum#81, count#82, count#83] Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] -(64) ColumnarToRow [codegen id : 10] +(69) CometColumnarToRow [codegen id : 10] Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -(65) HashAggregate [codegen id : 10] +(70) 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 : 10] +(71) 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 +(72) CometColumnarExchange +Input [4]: [sum#81, count#82, count#83, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] + +(73) CometColumnarToRow [codegen id : 11] Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] -(68) HashAggregate [codegen id : 11] +(74) HashAggregate [codegen id : 11] 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 +(75) BroadcastExchange Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] Arguments: IdentityBroadcastMode, [plan_id=17] -(70) BroadcastNestedLoopJoin [codegen id : 12] +(76) 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 4a547c4e04..e62a830d18 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 @@ -5,95 +5,101 @@ WholeStageCodegen (12) 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 (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 - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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 (3) 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #4 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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 (5) + 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] + CometColumnarToRow InputAdapter - Exchange #4 - WholeStageCodegen (2) + CometColumnarExchange #7 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #5 + CometExchange [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 #6 - WholeStageCodegen (5) - 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 + BroadcastExchange #9 + WholeStageCodegen (7) + 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] + CometColumnarToRow InputAdapter - Exchange #7 - WholeStageCodegen (4) + CometColumnarExchange #10 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #8 + CometExchange [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 #9 - WholeStageCodegen (7) - 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 + BroadcastExchange #12 + WholeStageCodegen (9) + 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] + CometColumnarToRow InputAdapter - Exchange #10 - WholeStageCodegen (6) + CometColumnarExchange #13 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #11 + CometExchange [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 #12 - WholeStageCodegen (9) - 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 + BroadcastExchange #15 + WholeStageCodegen (11) + 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] + CometColumnarToRow InputAdapter - Exchange #13 - WholeStageCodegen (8) + CometColumnarExchange #16 + 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 + CometColumnarToRow InputAdapter - CometExchange [ss_list_price] #14 + CometExchange [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] 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 (11) - 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 (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 - CometExchange [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] - 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/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 71d583785d..b02bf3a4cf 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -258,14 +258,14 @@ Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity# 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] +(45) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -286,7 +286,7 @@ Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#19] (50) BroadcastExchange @@ -295,7 +295,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 BroadcastExchange (55) -+- * ColumnarToRow (54) ++- * CometColumnarToRow (54) +- CometProject (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -316,7 +316,7 @@ Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 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] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] (55) BroadcastExchange @@ -325,7 +325,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 BroadcastExchange (60) -+- * ColumnarToRow (59) ++- * CometColumnarToRow (59) +- CometProject (58) +- CometFilter (57) +- CometScan parquet spark_catalog.default.date_dim (56) @@ -346,7 +346,7 @@ Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(59) ColumnarToRow [codegen id : 1] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (60) BroadcastExchange 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 9398fcdf5e..9340a4e5f1 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -24,7 +24,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -35,7 +35,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 a0796a47af..e80a2eaa78 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 f82fd24bdd..7bc02651ec 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 c0b9e17358..9930f6bbde 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -268,14 +268,14 @@ Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, 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] -(48) ColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [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 (53) -+- * ColumnarToRow (52) ++- * CometColumnarToRow (52) +- CometProject (51) +- CometFilter (50) +- CometScan parquet spark_catalog.default.date_dim (49) @@ -296,7 +296,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(52) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (53) BroadcastExchange 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 872bf51912..ee501650be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 76a62c8926..6ce03737eb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (90) +* CometColumnarToRow (90) +- CometSort (89) +- CometColumnarExchange (88) +- CometProject (87) @@ -504,14 +504,14 @@ Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENT 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] -(90) ColumnarToRow [codegen id : 1] +(90) CometColumnarToRow [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 (94) -+- * ColumnarToRow (93) ++- * CometColumnarToRow (93) +- CometFilter (92) +- CometScan parquet spark_catalog.default.date_dim (91) @@ -527,7 +527,7 @@ ReadSchema: struct Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(93) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (94) BroadcastExchange @@ -536,7 +536,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (98) -+- * ColumnarToRow (97) ++- * CometColumnarToRow (97) +- CometFilter (96) +- CometScan parquet spark_catalog.default.date_dim (95) @@ -552,7 +552,7 @@ ReadSchema: struct 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)) -(97) ColumnarToRow [codegen id : 1] +(97) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] (98) BroadcastExchange @@ -561,7 +561,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 BroadcastExchange (102) -+- * ColumnarToRow (101) ++- * CometColumnarToRow (101) +- CometFilter (100) +- CometScan parquet spark_catalog.default.date_dim (99) @@ -577,7 +577,7 @@ ReadSchema: struct 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)) -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] (102) BroadcastExchange 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 d6615fe1c1..22635bb9f2 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_qoy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] @@ -67,7 +67,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #12 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 03edc9859f..7caa3c6e24 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange 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 14b2a34a89..2650006a6c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 7a50fe69e9..587c91520a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 9556aa785b..50c2d64a92 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 50e3800526..452f25394a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 779d4cbddb..014786e7fd 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 c1e19555c9..74ff12aef1 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 9deb513423..8d4a8dcb97 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 5bd9122ffa..b10a21b708 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,7 +2,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- Window (27) - +- * ColumnarToRow (26) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometExchange (24) +- CometHashAggregate (23) @@ -148,7 +148,7 @@ Arguments: hashpartitioning(_w1#22, _w2#23, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [7]: [gross_margin#19, i_category#14, i_class#15, lochierarchy#20, _w0#21, _w1#22, _w2#23] (27) Window @@ -167,7 +167,7 @@ Arguments: 100, [lochierarchy#20 DESC NULLS LAST, CASE WHEN (lochierarchy#20 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(33) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (34) BroadcastExchange 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 421db8dad0..5b8623c2b3 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 844e7a729f..3f78c57bb2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-02-01)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 85e86ca8ae..13fcba8592 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 c0e7300dff..8139d06c08 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometHashAggregate (46) +- CometExchange (45) +- CometHashAggregate (44) @@ -261,14 +261,14 @@ Input [1]: [count#26] Keys: [] Functions [1]: [count(1)] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] (52) BroadcastExchange 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 8272260f5c..5ce896e6e6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] 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 32b231d65d..0b404d5868 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 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] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 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] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange 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 b789f3a8d2..9df61b1669 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [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 0239d76784..683ec549cc 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometSort (46) +- CometColumnarExchange (45) +- CometBroadcastHashJoin (44) @@ -261,14 +261,14 @@ Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC N 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -289,7 +289,7 @@ Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2 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] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_moy#11] (52) BroadcastExchange @@ -298,7 +298,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 BroadcastExchange (57) -+- * ColumnarToRow (56) ++- * CometColumnarToRow (56) +- CometProject (55) +- CometFilter (54) +- CometScan parquet spark_catalog.default.date_dim (53) @@ -319,7 +319,7 @@ Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2 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] -(56) ColumnarToRow [codegen id : 1] +(56) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#28, d_moy#30] (57) BroadcastExchange 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 b789f3a8d2..9df61b1669 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] @@ -52,7 +52,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [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 0fe37749f9..457cfd8a39 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (101) +* CometColumnarToRow (101) +- CometTakeOrderedAndProject (100) +- CometProject (99) +- CometBroadcastHashJoin (98) @@ -575,14 +575,14 @@ Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, custo 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] -(101) ColumnarToRow [codegen id : 1] +(101) CometColumnarToRow [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 (105) -+- * ColumnarToRow (104) ++- * CometColumnarToRow (104) +- CometFilter (103) +- CometScan parquet spark_catalog.default.date_dim (102) @@ -598,7 +598,7 @@ ReadSchema: struct 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)) -(104) ColumnarToRow [codegen id : 1] +(104) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_year#17] (105) BroadcastExchange @@ -607,7 +607,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 BroadcastExchange (109) -+- * ColumnarToRow (108) ++- * CometColumnarToRow (108) +- CometFilter (107) +- CometScan parquet spark_catalog.default.date_dim (106) @@ -623,7 +623,7 @@ ReadSchema: struct 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)) -(108) ColumnarToRow [codegen id : 1] +(108) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] (109) BroadcastExchange 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 d6651fadcf..19f7ccde3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -50,7 +50,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index ef66f763d2..10640eb2bb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometTakeOrderedAndProject (31) +- CometHashAggregate (30) +- CometExchange (29) @@ -179,14 +179,14 @@ Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - 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] +(32) CometColumnarToRow [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 (36) -+- * ColumnarToRow (35) ++- * CometColumnarToRow (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -202,7 +202,7 @@ ReadSchema: struct 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)) -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#16, d_date#17] (36) BroadcastExchange 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 eac6138196..6b87d9b8b0 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 48a1c60d16..bb3c74daeb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -103,6 +103,6 @@ Functions: [] 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] +(19) CometColumnarToRow [codegen id : 1] Input [1]: [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 007103499b..de12ce9581 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name] CometHashAggregate [i_product_name] 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 b58d82c785..0ad962d86a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 40528f52d3..7307450677 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 2953a5f8f9..10dfceddc2 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_s 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] +(19) CometColumnarToRow [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 447131b776..08b394e62e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 6e85991e4a..956ca43dbf 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject (33) : : : +- * Project (13) : : : +- * Filter (12) : : : +- Window (11) - : : : +- * ColumnarToRow (10) + : : : +- * CometColumnarToRow (10) : : : +- CometSort (9) : : : +- CometExchange (8) : : : +- CometFilter (7) @@ -24,11 +24,11 @@ TakeOrderedAndProject (33) : : +- * Project (20) : : +- * Filter (19) : : +- Window (18) - : : +- * ColumnarToRow (17) + : : +- * CometColumnarToRow (17) : : +- CometSort (16) : : +- ReusedExchange (15) : +- BroadcastExchange (27) - : +- * ColumnarToRow (26) + : +- * CometColumnarToRow (26) : +- CometFilter (25) : +- CometScan parquet spark_catalog.default.item (24) +- ReusedExchange (30) @@ -75,7 +75,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] 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] +(10) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] (11) Window @@ -101,7 +101,7 @@ Output [2]: [item_sk#12, rank_col#13] Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [2]: [item_sk#12, rank_col#13] (18) Window @@ -141,7 +141,7 @@ ReadSchema: struct Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(26) ColumnarToRow [codegen id : 5] +(26) CometColumnarToRow [codegen id : 5] Input [2]: [i_item_sk#15, i_product_name#16] (27) BroadcastExchange @@ -178,7 +178,7 @@ Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#19, worst_per ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (40) +* CometColumnarToRow (40) +- CometHashAggregate (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -216,7 +216,7 @@ Input [3]: [ss_store_sk#22, sum#25, count#26] Keys [1]: [ss_store_sk#22] Functions [1]: [avg(UnscaledValue(ss_net_profit#23))] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [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 6ae18d8792..3230b1456a 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 @@ -14,14 +14,14 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Window [rank_col] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] CometExchange #1 CometFilter [item_sk,rank_col] Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #3 @@ -43,14 +43,14 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] InputAdapter Window [rank_col] WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [item_sk,rank_col] ReusedExchange [item_sk,rank_col] #1 InputAdapter BroadcastExchange #4 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [i_item_sk,i_product_name] CometScan parquet spark_catalog.default.item [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 962a51203b..d785f36b6a 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,41 @@ == 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 (37) ++- * HashAggregate (36) + +- * CometColumnarToRow (35) + +- CometColumnarExchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) (1) CometScan parquet spark_catalog.default.web_sales @@ -149,7 +150,7 @@ Arguments: [ws_item_sk#2], [i_item_sk#15], Inner, BuildRight Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#15, i_item_id#16] Arguments: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16], [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#16] (25) CometScan parquet spark_catalog.default.item @@ -167,7 +168,7 @@ Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) Input [2]: [i_item_sk#17, i_item_id#18] Arguments: [i_item_id#18], [i_item_id#18] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [i_item_id#18] (29) BroadcastExchange @@ -195,50 +196,53 @@ 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) CometColumnarExchange Input [3]: [ca_zip#11, ca_city#10, sum#20] -Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(35) HashAggregate [codegen id : 3] +(35) CometColumnarToRow [codegen id : 3] +Input [3]: [ca_zip#11, ca_city#10, sum#20] + +(36) 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 +(37) 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 (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) -(37) CometScan parquet spark_catalog.default.date_dim +(38) CometScan 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 +(39) 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 +(40) 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] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#12] -(41) BroadcastExchange +(42) 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 f60fdb18a1..f43d2a0146 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,52 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 7bc10ce88e..406d3a6730 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#25, c_first_name#24, ca_city#30, bought_city#26, ss_tick 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] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (43) BroadcastExchange 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 e1b53bf4d0..25e38a548e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] 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 32126320ff..e39a63cc69 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [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] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 cf4556774d..26c6cd226a 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 70c7dc75f5..451f4d9646 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometHashAggregate (27) +- CometExchange (26) +- CometHashAggregate (25) @@ -159,14 +159,14 @@ Input [1]: [sum#18] Keys: [] Functions [1]: [sum(ss_quantity#4)] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [sum(ss_quantity)#19] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -187,7 +187,7 @@ Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#16, d_year#17] Arguments: [d_date_sk#16], [d_date_sk#16] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (33) BroadcastExchange 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 a46d8d6037..60c611beca 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] CometExchange #1 @@ -17,7 +17,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 2def4544c3..041046c4af 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,82 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * ColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (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) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (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) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (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) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (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) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (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) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (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) CometScan parquet spark_catalog.default.web_sales @@ -172,7 +173,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] 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) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +274,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] 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) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +375,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] 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) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -406,50 +407,51 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) Exchange +(75) 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, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 11] +(76) 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#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(77) TakeOrderedAndProject +(77) 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] + +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: 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] ===== 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 (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) -(78) CometScan parquet spark_catalog.default.date_dim +(79) CometScan 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 +(80) 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] 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-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index d98b2b0a66..21827c75d8 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,109 +1,110 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (11) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - 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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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 +WholeStageCodegen (11) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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 02ad2f3575..84d57d5645 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,75 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Expand (66) - +- Union (65) - :- * HashAggregate (22) - : +- * ColumnarToRow (21) - : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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 (71) ++- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (22) + : +- * CometColumnarToRow (21) + : +- CometExchange (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) + : +- * CometColumnarToRow (40) + : +- CometExchange (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) + +- * CometColumnarToRow (63) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -168,7 +169,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -259,7 +260,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled 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, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -369,7 +370,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -392,50 +393,53 @@ Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(prof 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) 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, [plan_id=4] +Arguments: hashpartitioning(channel#118, id#119, spark_grouping_id#120, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(69) HashAggregate [codegen id : 5] +(69) CometColumnarToRow [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] + +(70) 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 +(71) 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 (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) -(71) CometScan parquet spark_catalog.default.date_dim +(72) CometScan 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 +(73) 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 +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(74) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(75) BroadcastExchange +(76) 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 c6f9f60af4..7458de143c 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,93 +1,94 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - CometExchange [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] - 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 (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 - CometExchange [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] - 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 (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 - CometExchange [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] - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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) + CometColumnarToRow + 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 - 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] + 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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index 9b1efc7ada..ded5068bb0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -160,14 +160,14 @@ Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30 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] +(28) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -188,7 +188,7 @@ Condition : ((((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2 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] +(32) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#23] (33) BroadcastExchange 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 5333344640..c0a0b6d39c 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 8779bd70e7..cdfe36d00c 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,45 @@ == 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) - : +- * ColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (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) - +- * ColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * Project (16) + : +- Window (15) + : +- * CometColumnarToRow (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (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 (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan parquet spark_catalog.default.web_sales @@ -103,7 +104,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] (15) Window @@ -114,13 +115,13 @@ Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FI 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 +(17) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) Sort [codegen id : 3] +(18) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] (19) CometScan parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] @@ -168,84 +169,86 @@ Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeS 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] -(29) ColumnarToRow [codegen id : 4] +(29) CometColumnarToRow [codegen id : 3] Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] (30) 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] -(31) Project [codegen id : 5] +(31) 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] -(32) Exchange +(32) CometColumnarExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(33) Sort [codegen id : 6] +(33) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(34) SortMergeJoin [codegen id : 7] -Left keys [2]: [item_sk#9, d_date#6] -Right keys [2]: [item_sk#19, d_date#17] -Join type: FullOuter -Join condition: None +(34) 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 -(35) Project [codegen id : 7] -Output [4]: [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) 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] -(36) Exchange +(36) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) Sort [codegen id : 8] +(37) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], false, 0 +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] -(38) Window +(38) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] + +(39) Window Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [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] -(39) Filter [codegen id : 9] +(40) Filter [codegen id : 6] 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)) -(40) TakeOrderedAndProject +(41) TakeOrderedAndProject Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] Arguments: 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] ===== 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 (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) -(41) CometScan parquet spark_catalog.default.date_dim +(42) CometScan 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 +(43) 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 +(44) 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] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(45) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] 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-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index f972c31c17..c829621bcc 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,68 +1,62 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (9) + WholeStageCodegen (6) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (8) - Sort [item_sk,d_date] + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (7) - Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] - SortMergeJoin [item_sk,d_date,item_sk,d_date] - InputAdapter - WholeStageCodegen (3) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] - CometExchange [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] - InputAdapter - WholeStageCodegen (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #7 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (4) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] - CometExchange [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 + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [item_sk,d_date,_w0,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometExchange [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) + CometColumnarToRow + 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 + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [item_sk,d_date,_w0,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometExchange [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 42b974e53f..c7a400159f 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 af5223b69e..1e7168862a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 9b71fa4009..c1448cf812 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,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ 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) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 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] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] (34) BroadcastExchange 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 363b84b354..96a2eec50e 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 @@ -5,7 +5,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter Window [_w0,i_manufact_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_qoy] CometFilter [d_date_sk,d_month_seq,d_qoy] 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 73422b2923..72ba214250 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (56) +* CometColumnarToRow (56) +- CometTakeOrderedAndProject (55) +- CometHashAggregate (54) +- CometExchange (53) @@ -320,14 +320,14 @@ Functions [1]: [count(1)] 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] +(56) CometColumnarToRow [codegen id : 1] Input [3]: [segment#38, num_customers#40, segment_base#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (61) -+- * ColumnarToRow (60) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometFilter (58) +- CometScan parquet spark_catalog.default.date_dim (57) @@ -348,7 +348,7 @@ Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12 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] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] (61) BroadcastExchange @@ -359,7 +359,7 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dy Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 BroadcastExchange (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -380,7 +380,7 @@ Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= Subquery scalar- Input [2]: [d_date_sk#31, d_month_seq#32] Arguments: [d_date_sk#31], [d_date_sk#31] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#31] (66) BroadcastExchange @@ -388,7 +388,7 @@ Input [1]: [d_date_sk#31] 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#33, [id=#34] -* ColumnarToRow (73) +* CometColumnarToRow (73) +- CometHashAggregate (72) +- CometExchange (71) +- CometHashAggregate (70) @@ -426,11 +426,11 @@ Input [1]: [(d_month_seq + 1)#45] Keys [1]: [(d_month_seq + 1)#45] Functions: [] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [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) +* CometColumnarToRow (80) +- CometHashAggregate (79) +- CometExchange (78) +- CometHashAggregate (77) @@ -468,7 +468,7 @@ Input [1]: [(d_month_seq + 3)#49] Keys [1]: [(d_month_seq + 3)#49] Functions: [] -(80) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [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] 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 1290186685..e41bbe85b2 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] CometHashAggregate [segment,num_customers,segment_base,count,count(1)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -58,13 +58,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 1)] CometExchange [(d_month_seq + 1)] #10 @@ -74,7 +74,7 @@ WholeStageCodegen (1) CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [(d_month_seq + 3)] CometExchange [(d_month_seq + 3)] #11 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 46240a3c02..4549080d86 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (19) +* CometColumnarToRow (19) +- CometTakeOrderedAndProject (18) +- CometHashAggregate (17) +- CometExchange (16) @@ -106,6 +106,6 @@ Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] 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] +(19) CometColumnarToRow [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 999c8a6c4c..5a5f7ee21e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 c56eb8287f..e3977ede3f 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 2603a2781f..95ff3d709a 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 78df07e8fb..8f45480fc8 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, cc_name# Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 39a7ce9d67..10cd515da7 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index cd1e112f69..18736927ab 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (51) +* CometColumnarToRow (51) +- CometTakeOrderedAndProject (50) +- CometProject (49) +- CometBroadcastHashJoin (48) @@ -280,14 +280,14 @@ Arguments: [item_id#11, ss_item_rev#12, ss_dev#33, cs_item_rev#22, cs_dev#34, ws 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] +(51) CometColumnarToRow [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) ++- * CometColumnarToRow (60) +- CometProject (59) +- CometBroadcastHashJoin (58) :- CometFilter (53) @@ -337,7 +337,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(60) ColumnarToRow [codegen id : 1] +(60) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (61) BroadcastExchange @@ -345,7 +345,7 @@ Input [1]: [d_date_sk#7] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* ColumnarToRow (65) +* CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -366,7 +366,7 @@ Condition : (isnotnull(d_date#40) AND (d_date#40 = 2000-01-03)) Input [2]: [d_date#40, d_week_seq#41] Arguments: [d_week_seq#41], [d_week_seq#41] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#41] Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 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 36fc8ac573..db48fd9775 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] @@ -30,7 +30,7 @@ WholeStageCodegen (1) CometFilter [d_date,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_date,d_week_seq] 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 acdbfd1716..1091631a21 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,6 +219,6 @@ Arguments: [s_store_name1#26, s_store_id1#28, d_week_seq1#27, (sun_sales1 / sun_ 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] -(39) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [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 140a764391..15f2e45693 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 e87f6ce763..542920b01c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) 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] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [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 05566cbca5..1ba3d55d23 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 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 0d64004d9a..5596623c61 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (61) +* CometColumnarToRow (61) +- CometTakeOrderedAndProject (60) +- CometHashAggregate (59) +- CometExchange (58) @@ -339,14 +339,14 @@ Functions [1]: [sum(total_sales#36)] 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] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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 (66) -+- * ColumnarToRow (65) ++- * CometColumnarToRow (65) +- CometProject (64) +- CometFilter (63) +- CometScan parquet spark_catalog.default.date_dim (62) @@ -367,7 +367,7 @@ Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998 Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(65) ColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (66) BroadcastExchange 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 4dd8b3b5bd..ea973a4810 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 4a157e64e8..9656e93142 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,7 +1,7 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * ColumnarToRow (41) + :- * CometColumnarToRow (41) : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) @@ -43,7 +43,7 @@ : +- CometFilter (33) : +- CometScan parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * ColumnarToRow (62) + +- * CometColumnarToRow (62) +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) @@ -256,7 +256,7 @@ Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -(41) ColumnarToRow [codegen id : 2] +(41) CometColumnarToRow [codegen id : 2] Input [1]: [promotions#24] (42) CometScan parquet spark_catalog.default.store_sales @@ -345,7 +345,7 @@ Input [1]: [sum#37] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -(62) ColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [total#38] (63) BroadcastExchange @@ -364,7 +364,7 @@ Input [2]: [promotions#24, total#38] Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (70) -+- * ColumnarToRow (69) ++- * CometColumnarToRow (69) +- CometProject (68) +- CometFilter (67) +- CometScan parquet spark_catalog.default.date_dim (66) @@ -385,7 +385,7 @@ Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1 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] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (70) BroadcastExchange 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 548614416e..30c16b696d 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,7 +1,7 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 @@ -23,7 +23,7 @@ WholeStageCodegen (2) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -54,7 +54,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 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 c174784d38..b80eab371d 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) TH 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] +(28) CometColumnarToRow [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 6a9b1ced35..002eedf08e 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 2a5ae3d1f4..08e1a99a58 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,7 +3,7 @@ TakeOrderedAndProject (29) +- * Project (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,7 +144,7 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] (26) Window @@ -167,7 +167,7 @@ Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 BroadcastExchange (34) -+- * ColumnarToRow (33) ++- * CometColumnarToRow (33) +- CometProject (32) +- CometFilter (31) +- CometScan parquet spark_catalog.default.date_dim (30) @@ -188,7 +188,7 @@ Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207 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] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] (34) BroadcastExchange 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 8ea16ea740..8e1ce8c997 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 @@ -5,7 +5,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 @@ -27,7 +27,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_month_seq,d_moy] 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 2dcffc7a77..8fd7206781 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC 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] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] 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 (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct 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)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct 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)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange 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 49a08d6f8b..0ce55b5714 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index e0f1b47992..09ddb55e37 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometProject (37) +- CometBroadcastHashJoin (36) @@ -219,14 +219,14 @@ Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wh 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] +(39) CometColumnarToRow [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 = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -247,7 +247,7 @@ Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_mont Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] (44) BroadcastExchange 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 be08ee23a0..28412503a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 693eb5662d..58f093ae40 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometTakeOrderedAndProject (49) +- CometHashAggregate (48) +- CometExchange (47) @@ -277,14 +277,14 @@ Functions [36]: [sum(jan_sales#141), sum(feb_sales#142), sum(mar_sales#143), sum 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] +(50) CometColumnarToRow [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 (54) -+- * ColumnarToRow (53) ++- * CometColumnarToRow (53) +- CometFilter (52) +- CometScan parquet spark_catalog.default.date_dim (51) @@ -300,7 +300,7 @@ ReadSchema: struct 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)) -(53) ColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#16, d_year#17, d_moy#18] (54) BroadcastExchange 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 a2b5963e13..3529a210a6 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 4e78dad21d..487789ad10 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,7 +2,7 @@ TakeOrderedAndProject (28) +- * Filter (27) +- Window (26) - +- * ColumnarToRow (25) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -143,7 +143,7 @@ Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeS 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) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 @@ -162,7 +162,7 @@ Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_mont 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (33) BroadcastExchange 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 4825ab898b..3a51784865 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometExchange [i_category] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] 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 121987260e..00c0db86ec 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometProject (36) +- CometBroadcastHashJoin (35) @@ -213,14 +213,14 @@ Arguments: [c_last_name#27, c_first_name#26, ca_city#33, bought_city#28, ss_tick 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] -(38) ColumnarToRow [codegen id : 1] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -241,7 +241,7 @@ Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AN Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (43) BroadcastExchange 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 568fd734b9..986918c8b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 5386a3d166..d1080a5599 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer_address (28) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] Right output [1]: [ss_customer_sk#4] Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#10] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#14] (25) BroadcastExchange @@ -186,7 +187,7 @@ Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) Input [2]: [ca_address_sk#18, ca_state#19] Arguments: [ca_address_sk#18], [ca_address_sk#18] -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [1]: [ca_address_sk#18] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(49) BroadcastExchange +(50) 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 c9cc4959b7..09ef18a99b 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) - 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_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] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + CometColumnarToRow + 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) + CometColumnarToRow + 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] 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 d18a60b0b4..adc27c2e86 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometHashAggregate (28) +- CometExchange (27) @@ -169,14 +169,14 @@ Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(Uns 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] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [i_item_id#17, agg1#29, agg2#30, agg3#31, agg4#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (35) BroadcastExchange 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 044813ad42..96e7e32f95 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 e649f9e886..edc4ebf2ec 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,50 @@ == 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) - +- * ColumnarToRow (27) - +- CometSort (26) - +- CometHashAggregate (25) - +- CometExchange (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 (46) ++- * Project (45) + +- Window (44) + +- * CometColumnarToRow (43) + +- CometSort (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * Expand (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (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) + :- * CometColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.store (10) + +- BroadcastExchange (31) + +- * Project (30) + +- * Filter (29) + +- Window (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometHashAggregate (25) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -85,7 +87,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 : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -99,7 +101,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 : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -168,7 +170,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] 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] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) Window @@ -218,66 +220,72 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#23] Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] -(38) Exchange +(38) 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, [plan_id=4] +Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) HashAggregate [codegen id : 5] +(39) CometColumnarToRow [codegen id : 5] +Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] + +(40) 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))#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] -(40) Exchange +(41) 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] + +(42) CometSort 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, [plan_id=5] +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 : 6] +(43) CometColumnarToRow [codegen id : 6] Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] -Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0 -(42) Window +(44) 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 : 7] +(45) 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 +(46) 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 (51) ++- * CometColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.date_dim (47) -(45) CometScan parquet spark_catalog.default.date_dim +(47) CometScan 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 +(48) 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 +(49) 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] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) 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-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index 704213148f..85ad5d3b0e 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 @@ -4,66 +4,68 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (6) - Sort [_w1,_w2,_w0] + CometColumnarToRow InputAdapter - Exchange [_w1,_w2] #1 - WholeStageCodegen (5) - 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 (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 - 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 (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 #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))] - CometExchange [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 + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + 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) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometExchange [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 79a71af03a..fe8f7c5f76 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (37) +* CometColumnarToRow (37) +- CometSort (36) +- CometColumnarExchange (35) +- CometHashAggregate (34) @@ -209,14 +209,14 @@ Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS 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) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [5]: [brand_id#39, brand#40, t_hour#35, t_minute#36, ext_price#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (42) -+- * ColumnarToRow (41) ++- * CometColumnarToRow (41) +- CometProject (40) +- CometFilter (39) +- CometScan parquet spark_catalog.default.date_dim (38) @@ -237,7 +237,7 @@ Condition : ((((isnotnull(d_moy#12) AND isnotnull(d_year#11)) AND (d_moy#12 = 11 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] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (42) BroadcastExchange 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 88ca8b67df..7214ec67a5 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometColumnarExchange [ext_price,brand_id] #1 @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 6aae88fff3..4179f3b1d6 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 b0ca5520f2..a7c93664a7 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, Co 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 4417365325..eaede7232d 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometColumnarExchange [cnt] #1 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 c7a559c8ef..6358cf7e31 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange 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 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 2f33f50a8b..9d68c7a66c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[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]), [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], 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] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct 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)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct 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)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange 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 9e340130a5..5a8af58abc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index a2a17ccfce..24ed6809af 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (34) +* CometColumnarToRow (34) +- CometTakeOrderedAndProject (33) +- CometHashAggregate (32) +- CometExchange (31) @@ -190,6 +190,6 @@ Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] 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] +(34) CometColumnarToRow [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 eb3e14d5ec..3e89713523 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 661b23c47b..55f5778124 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,88 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (84) -+- * HashAggregate (83) - +- Exchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * ColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (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) - : +- CometExchange (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) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * ColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * Expand (80) + +- Union (79) + :- * CometColumnarToRow (31) + : +- CometProject (30) + : +- CometBroadcastHashJoin (29) + : :- CometHashAggregate (16) + : : +- CometExchange (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) + : +- CometExchange (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) + : : +- * CometColumnarToRow (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : +- ReusedExchange (33) + : +- * CometColumnarToRow (48) + : +- CometHashAggregate (47) + : +- CometExchange (46) + : +- CometHashAggregate (45) + : +- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : +- ReusedExchange (42) + +- * CometColumnarToRow (78) + +- CometProject (77) + +- CometBroadcastHashJoin (76) + :- CometHashAggregate (63) + : +- CometExchange (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) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -226,7 +227,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight 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] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -262,7 +263,7 @@ 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))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -302,7 +303,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -438,7 +439,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight 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] -(78) ColumnarToRow [codegen id : 4] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] (79) Union @@ -454,50 +455,53 @@ Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(prof 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] -(82) Exchange +(82) 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, [plan_id=8] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(83) HashAggregate [codegen id : 6] +(83) CometColumnarToRow [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] + +(84) 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] -(84) TakeOrderedAndProject +(85) TakeOrderedAndProject 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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * ColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan parquet spark_catalog.default.date_dim (85) +BroadcastExchange (90) ++- * CometColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan parquet spark_catalog.default.date_dim (86) -(85) CometScan parquet spark_catalog.default.date_dim +(86) CometScan 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 -(86) CometFilter +(87) 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)) -(87) CometProject +(88) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(88) ColumnarToRow [codegen id : 1] +(89) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(89) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#6] 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-v1_4/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt index e8226ab1fa..d9fa7d8f2e 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,110 +1,111 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - 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))] - CometExchange [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] - 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] - 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))] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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) + CometColumnarToRow 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))] - CometExchange [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 - ColumnarToRow + 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))] + CometExchange [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] + 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) + CometColumnarToRow + 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] + 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))] + CometExchange [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 - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #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 (4) - 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))] - CometExchange [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] - 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] - 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))] - CometExchange [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] + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [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 + CometColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #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 (4) + CometColumnarToRow + 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))] + CometExchange [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] + 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 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk] #12 + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + 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))] + CometExchange [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 b8b24b9d0b..41ea1060b6 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,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss 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] +(65) CometColumnarToRow [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] @@ -375,7 +375,7 @@ Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 D Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct 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)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange 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 3318702eff..6b85e03ae9 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,7 +1,7 @@ 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 (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - ColumnarToRow + CometColumnarToRow InputAdapter 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] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index e697c9038b..c4b54c4ed5 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometTakeOrderedAndProject (29) +- CometProject (28) +- CometBroadcastHashJoin (27) @@ -169,14 +169,14 @@ Arguments: [c_last_name#23, c_first_name#22, substr(s_city, 1, 30)#26, ss_ticket 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] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -197,7 +197,7 @@ Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,20 Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (35) BroadcastExchange 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 b69b0cd2e9..d5514ba8a7 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dow] 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 f02edb1912..c6f5a077fa 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (42) +* CometColumnarToRow (42) +- CometTakeOrderedAndProject (41) +- CometHashAggregate (40) +- CometExchange (39) @@ -236,14 +236,14 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] 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] +(42) CometColumnarToRow [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 (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -264,7 +264,7 @@ Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) A Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (47) BroadcastExchange 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 91d98aca9b..7beec478b3 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] 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 65fe1a28b0..d902d76cd2 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,106 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (102) -+- * HashAggregate (101) - +- Exchange (100) - +- * HashAggregate (99) - +- * Expand (98) - +- Union (97) - :- * HashAggregate (38) - : +- * ColumnarToRow (37) - : +- CometExchange (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) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (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) - : +- CometExchange (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) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (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) - +- CometExchange (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) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (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) +TakeOrderedAndProject (103) ++- * HashAggregate (102) + +- * CometColumnarToRow (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- * Expand (98) + +- Union (97) + :- * HashAggregate (38) + : +- * CometColumnarToRow (37) + : +- CometExchange (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) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (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) + : +- * CometColumnarToRow (66) + : +- CometExchange (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) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (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) + +- * CometColumnarToRow (95) + +- CometExchange (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) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -272,7 +273,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -403,7 +404,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c 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, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [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] @@ -534,7 +535,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c 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, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -557,50 +558,53 @@ Functions [3]: [partial_sum(sales#30), partial_sum(returns#31), partial_sum(prof 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) Exchange +(100) 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, [plan_id=10] +Arguments: hashpartitioning(channel#97, id#98, spark_grouping_id#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(101) HashAggregate [codegen id : 5] +(101) CometColumnarToRow [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] + +(102) 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] -(102) TakeOrderedAndProject +(103) 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 (107) -+- * ColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan parquet spark_catalog.default.date_dim (103) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(103) CometScan parquet spark_catalog.default.date_dim +(104) CometScan 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 -(104) CometFilter +(105) 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)) -(105) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(106) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(107) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 012c711bfe..981b9671d1 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,122 +1,123 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 - CometExchange [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] - CometExchange [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] - CometExchange [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 - CometExchange [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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id,spark_grouping_id] #1 + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 8ae0dae9ac..86f7932331 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (47) +* CometColumnarToRow (47) +- CometTakeOrderedAndProject (46) +- CometProject (45) +- CometBroadcastHashJoin (44) @@ -263,14 +263,14 @@ Arguments: [c_customer_id#21, c_salutation#23, c_first_name#24, c_last_name#25, 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] -(47) ColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [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 (52) -+- * ColumnarToRow (51) ++- * CometColumnarToRow (51) +- CometProject (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -291,7 +291,7 @@ Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(51) ColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (52) BroadcastExchange 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 b5d2c83218..4887a152c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 34319a8fc3..6117d4a921 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (26) +* CometColumnarToRow (26) +- CometTakeOrderedAndProject (25) +- CometHashAggregate (24) +- CometExchange (23) @@ -145,14 +145,14 @@ Functions: [] 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] +(26) CometColumnarToRow [codegen id : 1] Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 BroadcastExchange (31) -+- * ColumnarToRow (30) ++- * CometColumnarToRow (30) +- CometProject (29) +- CometFilter (28) +- CometScan parquet spark_catalog.default.date_dim (27) @@ -173,7 +173,7 @@ Condition : (((isnotnull(d_date#11) AND (d_date#11 >= 2000-05-25)) AND (d_date#1 Input [2]: [d_date_sk#10, d_date#11] Arguments: [d_date_sk#10], [d_date_sk#10] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (31) BroadcastExchange 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 42c3f40e68..755128d2c4 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] CometHashAggregate [i_item_id,i_item_desc,i_current_price] @@ -22,7 +22,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 c0dfa3fdf0..820bc9bd5c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (48) +* CometColumnarToRow (48) +- CometTakeOrderedAndProject (47) +- CometProject (46) +- CometBroadcastHashJoin (45) @@ -265,14 +265,14 @@ Arguments: [item_id#21, sr_item_qty#22, sr_dev#33, cr_item_qty#20, cr_dev#34, wr 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] +(48) CometColumnarToRow [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) ++- * CometColumnarToRow (61) +- CometProject (60) +- CometBroadcastHashJoin (59) :- CometFilter (50) @@ -344,7 +344,7 @@ Arguments: [d_date#8], [d_date#9], LeftSemi, BuildRight Input [2]: [d_date_sk#7, d_date#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (62) BroadcastExchange 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 77345d28a7..0530aad2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -18,7 +18,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometBroadcastHashJoin [d_date_sk,d_date,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt index 572fd7a66f..ca784ef3e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (32) +- * Project (31) - +- * ColumnarToRow (30) + +- * CometColumnarToRow (30) +- CometBroadcastHashJoin (29) :- CometBroadcastExchange (25) : +- CometProject (24) @@ -172,7 +172,7 @@ Left output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] Right output [1]: [sr_cdemo_sk#15] Arguments: [cd_demo_sk#9], [sr_cdemo_sk#15], Inner, BuildLeft -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] (31) Project [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index 646285a082..3101b29dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [c_customer_id,customer_id,customername] WholeStageCodegen (1) Project [c_customer_id,c_last_name,c_first_name] - ColumnarToRow + CometColumnarToRow InputAdapter CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 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 c03da66a07..020414c4d7 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (45) +* CometColumnarToRow (45) +- CometTakeOrderedAndProject (44) +- CometHashAggregate (43) +- CometExchange (42) @@ -256,14 +256,14 @@ Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg 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] +(45) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (50) -+- * ColumnarToRow (49) ++- * CometColumnarToRow (49) +- CometProject (48) +- CometFilter (47) +- CometScan parquet spark_catalog.default.date_dim (46) @@ -284,7 +284,7 @@ Condition : ((isnotnull(d_year#29) AND (d_year#29 = 2000)) AND isnotnull(d_date_ Input [2]: [d_date_sk#28, d_year#29] Arguments: [d_date_sk#28], [d_date_sk#28] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#28] (50) BroadcastExchange 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 4e4ece1bda..28752f3783 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -25,7 +25,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 cf40b3e8d5..e7ce31dd2b 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,7 +2,7 @@ TakeOrderedAndProject (23) +- * Project (22) +- Window (21) - +- * ColumnarToRow (20) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,7 +114,7 @@ Arguments: hashpartitioning(_w1#17, _w2#18, 5), ENSURE_REQUIREMENTS, CometNative 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) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#14, i_category#10, i_class#11, lochierarchy#15, _w0#16, _w1#17, _w2#18] (21) Window @@ -133,7 +133,7 @@ Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (28) -+- * ColumnarToRow (27) ++- * CometColumnarToRow (27) +- CometProject (26) +- CometFilter (25) +- CometScan parquet spark_catalog.default.date_dim (24) @@ -154,7 +154,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(27) ColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (28) BroadcastExchange 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 86480d58c6..b1bf4c2e38 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [_w0,_w1,_w2] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 @@ -21,7 +21,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 28b942e2f7..d496a7ecb2 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,54 @@ == Physical Plan == -* HashAggregate (49) -+- Exchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin LeftAnti BuildRight (45) - :- * BroadcastHashJoin LeftAnti BuildRight (31) - : :- * ColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometExchange (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) - : +- CometExchange (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) - +- CometExchange (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 (50) ++- * CometColumnarToRow (49) + +- CometColumnarExchange (48) + +- * HashAggregate (47) + +- * Project (46) + +- * BroadcastHashJoin LeftAnti BuildRight (45) + :- * BroadcastHashJoin LeftAnti BuildRight (31) + : :- * CometColumnarToRow (17) + : : +- CometHashAggregate (16) + : : +- CometExchange (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) + : +- * CometColumnarToRow (29) + : +- CometHashAggregate (28) + : +- CometExchange (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) + +- * CometColumnarToRow (43) + +- CometHashAggregate (42) + +- CometExchange (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) CometScan parquet spark_catalog.default.store_sales @@ -128,7 +129,7 @@ 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: [] -(17) ColumnarToRow [codegen id : 3] +(17) CometColumnarToRow [codegen id : 3] Input [3]: [c_last_name#9, c_first_name#8, d_date#5] (18) CometScan parquet spark_catalog.default.catalog_sales @@ -181,7 +182,7 @@ 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: [] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [3]: [c_last_name#17, c_first_name#16, d_date#14] (30) BroadcastExchange @@ -244,7 +245,7 @@ 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: [] -(43) ColumnarToRow [codegen id : 2] +(43) CometColumnarToRow [codegen id : 2] Input [3]: [c_last_name#25, c_first_name#24, d_date#22] (44) BroadcastExchange @@ -268,11 +269,14 @@ Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#26] Results [1]: [count#27] -(48) Exchange +(48) CometColumnarExchange Input [1]: [count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) HashAggregate [codegen id : 4] +(49) CometColumnarToRow [codegen id : 4] +Input [1]: [count#27] + +(50) HashAggregate [codegen id : 4] Input [1]: [count#27] Keys: [] Functions [1]: [count(1)] @@ -282,32 +286,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 (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(50) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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 +(53) 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] +(54) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#4, d_date#5] -(54) BroadcastExchange +(55) 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 6e3328ae1d..e4f27f3a1b 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,69 +1,70 @@ WholeStageCodegen (4) HashAggregate [count] [count(1),count(1),count] - InputAdapter - Exchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (3) + HashAggregate [count,count] + Project BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - ColumnarToRow + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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) + CometColumnarToRow + 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 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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] + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [c_last_name,c_first_name,d_date] + CometExchange [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 InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #8 + WholeStageCodegen (2) + CometColumnarToRow InputAdapter CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 + CometExchange [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 [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] + 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 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 592e23cd27..6682e151f1 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,7 +6,7 @@ : : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) : : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * ColumnarToRow (25) +: : : : : : :- * CometColumnarToRow (25) : : : : : : : +- CometHashAggregate (24) : : : : : : : +- CometExchange (23) : : : : : : : +- CometHashAggregate (22) @@ -32,7 +32,7 @@ : : : : : : : +- CometFilter (17) : : : : : : : +- CometScan parquet spark_catalog.default.store (16) : : : : : : +- BroadcastExchange (45) -: : : : : : +- * ColumnarToRow (44) +: : : : : : +- * CometColumnarToRow (44) : : : : : : +- CometHashAggregate (43) : : : : : : +- CometExchange (42) : : : : : : +- CometHashAggregate (41) @@ -52,7 +52,7 @@ : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (32) : : : : : : +- ReusedExchange (38) : : : : : +- BroadcastExchange (66) -: : : : : +- * ColumnarToRow (65) +: : : : : +- * CometColumnarToRow (65) : : : : : +- CometHashAggregate (64) : : : : : +- CometExchange (63) : : : : : +- CometHashAggregate (62) @@ -72,7 +72,7 @@ : : : : : : +- CometScan parquet spark_catalog.default.time_dim (53) : : : : : +- ReusedExchange (59) : : : : +- BroadcastExchange (87) -: : : : +- * ColumnarToRow (86) +: : : : +- * CometColumnarToRow (86) : : : : +- CometHashAggregate (85) : : : : +- CometExchange (84) : : : : +- CometHashAggregate (83) @@ -92,7 +92,7 @@ : : : : : +- CometScan parquet spark_catalog.default.time_dim (74) : : : : +- ReusedExchange (80) : : : +- BroadcastExchange (108) -: : : +- * ColumnarToRow (107) +: : : +- * CometColumnarToRow (107) : : : +- CometHashAggregate (106) : : : +- CometExchange (105) : : : +- CometHashAggregate (104) @@ -112,7 +112,7 @@ : : : : +- CometScan parquet spark_catalog.default.time_dim (95) : : : +- ReusedExchange (101) : : +- BroadcastExchange (129) -: : +- * ColumnarToRow (128) +: : +- * CometColumnarToRow (128) : : +- CometHashAggregate (127) : : +- CometExchange (126) : : +- CometHashAggregate (125) @@ -132,7 +132,7 @@ : : : +- CometScan parquet spark_catalog.default.time_dim (116) : : +- ReusedExchange (122) : +- BroadcastExchange (150) -: +- * ColumnarToRow (149) +: +- * CometColumnarToRow (149) : +- CometHashAggregate (148) : +- CometExchange (147) : +- CometHashAggregate (146) @@ -152,7 +152,7 @@ : : +- CometScan parquet spark_catalog.default.time_dim (137) : +- ReusedExchange (143) +- BroadcastExchange (171) - +- * ColumnarToRow (170) + +- * CometColumnarToRow (170) +- CometHashAggregate (169) +- CometExchange (168) +- CometHashAggregate (167) @@ -285,7 +285,7 @@ Input [1]: [count#13] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 8] +(25) CometColumnarToRow [codegen id : 8] Input [1]: [h8_30_to_9#14] (26) CometScan parquet spark_catalog.default.store_sales @@ -368,7 +368,7 @@ Input [1]: [count#24] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [h9_to_9_30#25] (45) BroadcastExchange @@ -459,7 +459,7 @@ Input [1]: [count#35] Keys: [] Functions [1]: [count(1)] -(65) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [codegen id : 2] Input [1]: [h9_30_to_10#36] (66) BroadcastExchange @@ -550,7 +550,7 @@ Input [1]: [count#46] Keys: [] Functions [1]: [count(1)] -(86) ColumnarToRow [codegen id : 3] +(86) CometColumnarToRow [codegen id : 3] Input [1]: [h10_to_10_30#47] (87) BroadcastExchange @@ -641,7 +641,7 @@ Input [1]: [count#57] Keys: [] Functions [1]: [count(1)] -(107) ColumnarToRow [codegen id : 4] +(107) CometColumnarToRow [codegen id : 4] Input [1]: [h10_30_to_11#58] (108) BroadcastExchange @@ -732,7 +732,7 @@ Input [1]: [count#68] Keys: [] Functions [1]: [count(1)] -(128) ColumnarToRow [codegen id : 5] +(128) CometColumnarToRow [codegen id : 5] Input [1]: [h11_to_11_30#69] (129) BroadcastExchange @@ -823,7 +823,7 @@ Input [1]: [count#79] Keys: [] Functions [1]: [count(1)] -(149) ColumnarToRow [codegen id : 6] +(149) CometColumnarToRow [codegen id : 6] Input [1]: [h11_30_to_12#80] (150) BroadcastExchange @@ -914,7 +914,7 @@ Input [1]: [count#90] Keys: [] Functions [1]: [count(1)] -(170) ColumnarToRow [codegen id : 7] +(170) CometColumnarToRow [codegen id : 7] Input [1]: [h12_to_12_30#91] (171) BroadcastExchange 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 1bb61b6c92..21b72d36a7 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 @@ -6,7 +6,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin BroadcastNestedLoopJoin BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h8_30_to_9,count,count(1)] CometExchange #1 @@ -35,7 +35,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_to_9_30,count,count(1)] CometExchange #6 @@ -58,7 +58,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #8 WholeStageCodegen (2) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h9_30_to_10,count,count(1)] CometExchange #9 @@ -81,7 +81,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #11 WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_to_10_30,count,count(1)] CometExchange #12 @@ -104,7 +104,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #14 WholeStageCodegen (4) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h10_30_to_11,count,count(1)] CometExchange #15 @@ -127,7 +127,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #17 WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_to_11_30,count,count(1)] CometExchange #18 @@ -150,7 +150,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #20 WholeStageCodegen (6) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h11_30_to_12,count,count(1)] CometExchange #21 @@ -173,7 +173,7 @@ WholeStageCodegen (8) InputAdapter BroadcastExchange #23 WholeStageCodegen (7) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [h12_to_12_30,count,count(1)] CometExchange #24 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 143ff08683..8346f9b848 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,7 +3,7 @@ TakeOrderedAndProject (28) +- * Project (27) +- * Filter (26) +- Window (25) - +- * ColumnarToRow (24) + +- * CometColumnarToRow (24) +- CometSort (23) +- CometExchange (22) +- CometHashAggregate (21) @@ -139,7 +139,7 @@ Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#14, s_company_ 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) ColumnarToRow [codegen id : 1] +(24) CometColumnarToRow [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 @@ -162,7 +162,7 @@ Arguments: 100, [(sum_sales#17 - avg_monthly_sales#19) ASC NULLS FIRST, s_store_ Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 BroadcastExchange (33) -+- * ColumnarToRow (32) ++- * CometColumnarToRow (32) +- CometProject (31) +- CometFilter (30) +- CometScan parquet spark_catalog.default.date_dim (29) @@ -183,7 +183,7 @@ Condition : ((isnotnull(d_year#11) AND (d_year#11 = 1999)) AND isnotnull(d_date_ 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] +(32) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#10, d_moy#12] (33) BroadcastExchange 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 97b9563be6..58bb7e979e 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 @@ -5,7 +5,7 @@ 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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_moy] CometFilter [d_date_sk,d_year,d_moy] 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 8261f96537..831b71a98c 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 @@ -1,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (4) +* CometColumnarToRow (4) +- CometProject (3) +- CometFilter (2) +- CometScan parquet spark_catalog.default.reason (1) @@ -20,13 +20,13 @@ Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) Input [1]: [r_reason_sk#1] Arguments: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6], [CASE WHEN (Subquery scalar-subquery#7, [id=#8].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#7, [id=#8].avg(ss_net_paid) END AS bucket1#2, CASE WHEN (Subquery scalar-subquery#9, [id=#10].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#9, [id=#10].avg(ss_net_paid) END AS bucket2#3, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket3#4, CASE WHEN (Subquery scalar-subquery#13, [id=#14].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#13, [id=#14].avg(ss_net_paid) END AS bucket4#5, CASE WHEN (Subquery scalar-subquery#15, [id=#16].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#15, [id=#16].avg(ss_net_paid) END AS bucket5#6] -(4) ColumnarToRow [codegen id : 1] +(4) CometColumnarToRow [codegen id : 1] Input [5]: [bucket1#2, bucket2#3, bucket3#4, bucket4#5, bucket5#6] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#7, [id=#8] -* ColumnarToRow (12) +* CometColumnarToRow (12) +- CometProject (11) +- CometHashAggregate (10) +- CometExchange (9) @@ -69,7 +69,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(Unscal 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] +(12) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#29] Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] @@ -77,7 +77,7 @@ Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:3 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] Subquery:4 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* ColumnarToRow (20) +* CometColumnarToRow (20) +- CometProject (19) +- CometHashAggregate (18) +- CometExchange (17) @@ -120,7 +120,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#42] Subquery:5 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] @@ -128,7 +128,7 @@ Subquery:5 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:6 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] Subquery:7 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -171,7 +171,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(Unscal 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] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#55] Subquery:8 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] @@ -179,7 +179,7 @@ Subquery:8 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:9 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] Subquery:10 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#13, [id=#14] -* ColumnarToRow (36) +* CometColumnarToRow (36) +- CometProject (35) +- CometHashAggregate (34) +- CometExchange (33) @@ -222,7 +222,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#68] Subquery:11 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] @@ -230,7 +230,7 @@ Subquery:11 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery Subquery:12 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#13, [id=#14] Subquery:13 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#15, [id=#16] -* ColumnarToRow (44) +* CometColumnarToRow (44) +- CometProject (43) +- CometHashAggregate (42) +- CometExchange (41) @@ -273,7 +273,7 @@ Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(Unscal 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) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [mergedValue#81] Subquery:14 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#15, [id=#16] 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 cfeb9ebe74..3e1b996a51 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,10 +1,10 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [bucket1,bucket2,bucket3,bucket4,bucket5] Subquery #1 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -17,7 +17,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #1 Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -30,7 +30,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #2 Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -43,7 +43,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #3 Subquery #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] @@ -56,7 +56,7 @@ WholeStageCodegen (1) ReusedSubquery [mergedValue] #4 Subquery #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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))] 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 a6ec6f4b96..38379fb61c 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,7 +1,7 @@ == Physical Plan == * Project (47) +- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * ColumnarToRow (25) + :- * CometColumnarToRow (25) : +- CometHashAggregate (24) : +- CometExchange (23) : +- CometHashAggregate (22) @@ -27,7 +27,7 @@ : +- CometFilter (17) : +- CometScan parquet spark_catalog.default.web_page (16) +- BroadcastExchange (45) - +- * ColumnarToRow (44) + +- * CometColumnarToRow (44) +- CometHashAggregate (43) +- CometExchange (42) +- CometHashAggregate (41) @@ -160,7 +160,7 @@ Input [1]: [count#11] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 2] +(25) CometColumnarToRow [codegen id : 2] Input [1]: [amc#12] (26) CometScan parquet spark_catalog.default.web_sales @@ -243,7 +243,7 @@ Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -(44) ColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [pmc#22] (45) BroadcastExchange 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 95fd73d86e..72e45a9ad1 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,7 +1,7 @@ WholeStageCodegen (2) Project [amc,pmc] BroadcastNestedLoopJoin - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [amc,count,count(1)] CometExchange #1 @@ -30,7 +30,7 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [pmc,count,count(1)] CometExchange #6 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 ede0953066..26272decdb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (41) +* CometColumnarToRow (41) +- CometSort (40) +- CometColumnarExchange (39) +- CometHashAggregate (38) @@ -232,14 +232,14 @@ Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIRE 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) ColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 3 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 BroadcastExchange (46) -+- * ColumnarToRow (45) ++- * CometColumnarToRow (45) +- CometProject (44) +- CometFilter (43) +- CometScan parquet spark_catalog.default.date_dim (42) @@ -260,7 +260,7 @@ Condition : ((((isnotnull(d_year#11) AND isnotnull(d_moy#12)) AND (d_year#11 = 1 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] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] (46) BroadcastExchange 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 0dc197b1b4..1847051c7f 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometColumnarExchange [Returns_Loss] #1 @@ -26,7 +26,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 93bb03fae1..25045775b0 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (30) +* CometColumnarToRow (30) +- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) @@ -168,14 +168,14 @@ Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -(30) ColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [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 (35) -+- * ColumnarToRow (34) ++- * CometColumnarToRow (34) +- CometProject (33) +- CometFilter (32) +- CometScan parquet spark_catalog.default.date_dim (31) @@ -196,7 +196,7 @@ Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#1 Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#16] (35) BroadcastExchange 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 f49dd1ea30..4ed5b0b8ef 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 @@ -15,7 +15,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] 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 dc64f3c4ce..ae826e9288 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (22) +* CometColumnarToRow (22) +- CometTakeOrderedAndProject (21) +- CometHashAggregate (20) +- CometExchange (19) @@ -119,6 +119,6 @@ Functions [1]: [sum(act_sales#14)] 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] +(22) CometColumnarToRow [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 6795d7e399..9580e56671 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] 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 4dd9246cd7..54db114394 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,44 +1,45 @@ == Physical Plan == -* HashAggregate (40) -+- Exchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * ColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan parquet spark_catalog.default.web_site (29) +* HashAggregate (41) ++- * CometColumnarToRow (40) + +- CometColumnarExchange (39) + +- * HashAggregate (38) + +- * HashAggregate (37) + +- * CometColumnarToRow (36) + +- CometHashAggregate (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometSortMergeJoin (16) + : : : :- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : +- CometSort (15) + : : : +- CometExchange (14) + : : : +- CometProject (13) + : : : +- CometScan parquet spark_catalog.default.web_returns (12) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.date_dim (17) + : +- CometBroadcastExchange (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.customer_address (23) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.web_site (29) (1) CometScan parquet spark_catalog.default.web_sales @@ -203,7 +204,7 @@ 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))] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#5, sum#20, sum#21] (37) HashAggregate [codegen id : 1] @@ -220,11 +221,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] Results [3]: [sum#20, sum#21, count#25] -(39) Exchange +(39) CometColumnarExchange Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) HashAggregate [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#20, sum#21, count#25] + +(41) HashAggregate [codegen id : 2] Input [3]: [sum#20, sum#21, 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 601f577da9..9001c8e061 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,44 +1,45 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - 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] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometSortMergeJoin [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_order_number,ws_warehouse_sk] + 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] + CometExchange [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] + CometSort [ws_warehouse_sk,ws_order_number] + CometExchange [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] + CometSort [wr_order_number] + CometExchange [wr_order_number] #4 + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 2e951a408c..1d982a25cd 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,57 +1,58 @@ == Physical Plan == -* HashAggregate (53) -+- Exchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * ColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan parquet spark_catalog.default.web_site (42) +* HashAggregate (54) ++- * CometColumnarToRow (53) + +- CometColumnarExchange (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * CometColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometBroadcastHashJoin (46) + :- CometProject (41) + : +- CometBroadcastHashJoin (40) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometSortMergeJoin (29) + : : : :- CometSortMergeJoin (15) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- CometProject (14) + : : : : +- CometSortMergeJoin (13) + : : : : :- CometSort (10) + : : : : : +- CometExchange (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : : : +- CometSort (12) + : : : : +- ReusedExchange (11) + : : : +- CometProject (28) + : : : +- CometSortMergeJoin (27) + : : : :- CometSort (20) + : : : : +- CometExchange (19) + : : : : +- CometProject (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.web_returns (16) + : : : +- CometProject (26) + : : : +- CometSortMergeJoin (25) + : : : :- CometSort (22) + : : : : +- ReusedExchange (21) + : : : +- CometSort (24) + : : : +- ReusedExchange (23) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.date_dim (30) + : +- CometBroadcastExchange (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer_address (36) + +- CometBroadcastExchange (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.web_site (42) (1) CometScan parquet spark_catalog.default.web_sales @@ -270,7 +271,7 @@ 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))] -(49) ColumnarToRow [codegen id : 1] +(49) CometColumnarToRow [codegen id : 1] Input [3]: [ws_order_number#4, sum#21, sum#22] (50) HashAggregate [codegen id : 1] @@ -287,11 +288,14 @@ Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(Unscaled Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24, count(ws_order_number#4)#25] Results [3]: [sum#21, sum#22, count#26] -(52) Exchange +(52) CometColumnarExchange Input [3]: [sum#21, sum#22, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) HashAggregate [codegen id : 2] +(53) CometColumnarToRow [codegen id : 2] +Input [3]: [sum#21, sum#22, count#26] + +(54) HashAggregate [codegen id : 2] Input [3]: [sum#21, sum#22, 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 168f353a7b..b1aaa8fc44 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,57 +1,58 @@ WholeStageCodegen (2) 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 (1) - 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] - ColumnarToRow - InputAdapter - CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [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] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #1 + WholeStageCodegen (1) + 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] + CometColumnarToRow + InputAdapter + CometHashAggregate [ws_order_number,sum,sum,ws_ext_ship_cost,ws_net_profit] + CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] + CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] + CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] + CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] + CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + CometExchange [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] CometProject [ws_order_number] CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometExchange [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] CometSort [ws_warehouse_sk,ws_order_number] ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - 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 [wr_order_number] + CometSortMergeJoin [wr_order_number,ws_order_number] + CometSort [wr_order_number] + CometExchange [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] + CometProject [ws_order_number] + CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + CometSort [ws_warehouse_sk,ws_order_number] + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + 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 [ca_address_sk] #6 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + CometBroadcastExchange [web_site_sk] #7 + 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 d2e63bee25..677abdadae 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -138,6 +138,6 @@ Input [1]: [count#12] Keys: [] Functions [1]: [count(1)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 d9a87aa3c4..9e86a025ee 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [count(1),count,count(1)] CometExchange #1 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 1c6e9b78c0..f23cf90e66 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometHashAggregate (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -139,14 +139,14 @@ 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)] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [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 (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -167,7 +167,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (30) BroadcastExchange 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 0036a4bd84..aa5be145e9 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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)] CometExchange #1 @@ -16,7 +16,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 593065ba0b..8c2521b90c 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,11 +1,11 @@ == Physical Plan == -* ColumnarToRow (25) +* CometColumnarToRow (25) +- CometProject (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -112,7 +112,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -135,14 +135,14 @@ Arguments: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve 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] -(25) ColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [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 (30) -+- * ColumnarToRow (29) ++- * CometColumnarToRow (29) +- CometProject (28) +- CometFilter (27) +- CometScan parquet spark_catalog.default.date_dim (26) @@ -163,7 +163,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(29) ColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (30) BroadcastExchange 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 01aa117344..a77e84b60b 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,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow 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] @@ -9,7 +9,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 @@ -25,7 +25,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 6dfcf8b322..b4c43d5d01 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (28) +* CometColumnarToRow (28) +- CometTakeOrderedAndProject (27) +- CometHashAggregate (26) +- CometExchange (25) @@ -158,6 +158,6 @@ Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) TH 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] +(28) CometColumnarToRow [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 51599575db..e112972b12 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] 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 868b2f4822..c2b7f49205 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(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] 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] +(39) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange 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 2e8f2022cf..b570b700ac 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 689697306d..0bcd9798e8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, custo 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (76) BroadcastExchange 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 8743dd42ef..987c965c03 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 a428886298..b8d68b0a31 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 6f51be0c5b..4a829e8612 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 130726664c..a315b85659 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_c 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] +(85) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#96, count#97] Keys: [] Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#98] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] Arguments: [d_week_seq#99], [d_week_seq#99] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#99] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 < Input [2]: [d_date_sk#26, d_year#103] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subq Input [2]: [d_date_sk#65, d_week_seq#66] Arguments: [d_date_sk#65], [d_date_sk#65] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#65] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#65] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] Arguments: [d_week_seq#104], [d_week_seq#104] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#104] 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 bb4be4d3ae..befc6dadbb 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,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] 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 c5c211461a..9dd52b68f1 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (122) +* CometColumnarToRow (122) +- CometTakeOrderedAndProject (121) +- CometHashAggregate (120) +- CometExchange (119) @@ -682,13 +682,13 @@ Functions: [] 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] +(122) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (142) +* CometColumnarToRow (142) +- CometHashAggregate (141) +- CometExchange (140) +- CometHashAggregate (139) @@ -802,14 +802,14 @@ Input [2]: [sum#165, count#166] Keys: [] Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] -(142) ColumnarToRow [codegen id : 1] +(142) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#167] Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 BroadcastExchange (147) -+- * ColumnarToRow (146) ++- * CometColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 < Input [2]: [d_date_sk#154, d_year#155] Arguments: [d_date_sk#154], [d_date_sk#154] -(146) ColumnarToRow [codegen id : 1] +(146) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#154] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * ColumnarToRow (151) ++- * CometColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) ColumnarToRow [codegen id : 1] +(151) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * ColumnarToRow (156) ++- * CometColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 < Input [2]: [d_date_sk#26, d_year#168] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) ColumnarToRow [codegen id : 1] +(156) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (157) BroadcastExchange 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 760d0963ed..162f4f2447 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 @@ -30,7 +30,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -81,7 +81,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 e227c81b82..8eeb8487de 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (143) +* CometColumnarToRow (143) +- CometTakeOrderedAndProject (142) +- CometUnion (141) :- CometHashAggregate (38) @@ -785,14 +785,14 @@ Child 4 Input [11]: [i_item_id#256, ca_country#257, ca_state#258, county#259, ag 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] -(143) ColumnarToRow [codegen id : 1] +(143) CometColumnarToRow [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 (148) -+- * ColumnarToRow (147) ++- * CometColumnarToRow (147) +- CometProject (146) +- CometFilter (145) +- CometScan parquet spark_catalog.default.date_dim (144) @@ -813,7 +813,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(147) ColumnarToRow [codegen id : 1] +(147) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (148) BroadcastExchange 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 add76afad3..b974a7afce 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 a0c02e00e9..0610ff5ce3 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 c4d5e48ae5..8e639b6fa0 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 34d510fc71..1afbec79bb 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,29 @@ == 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 (25) ++- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastNestedLoopJoin Inner BuildRight (18) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (16) + +- CometScan parquet spark_catalog.default.warehouse (15) (1) CometScan parquet spark_catalog.default.inventory @@ -89,7 +90,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +99,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -124,50 +125,53 @@ 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) 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, [plan_id=2] +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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarToRow [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] + +(24) 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 +(25) 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 (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(25) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(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)) -(27) CometProject +(28) 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] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(30) 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 0e864ab701..7fed747fe6 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,40 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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 + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 deae565119..4457b74a5a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (46) +* CometColumnarToRow (46) +- CometTakeOrderedAndProject (45) +- CometUnion (44) :- CometHashAggregate (23) @@ -258,14 +258,14 @@ Child 4 Input [5]: [i_product_name#69, i_brand#70, i_class#71, i_category#72, qo 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) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#59] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan parquet spark_catalog.default.date_dim (47) @@ -286,7 +286,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (51) BroadcastExchange 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 63eda27a6b..02f615b128 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 522e028b5a..0e031ade47 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,49 +1,51 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometSort (44) - +- CometColumnarExchange (43) - +- * Filter (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* CometColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- * Filter (44) + +- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -177,7 +179,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] +(29) CometColumnarToRow [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] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +193,7 @@ ReadSchema: struct cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(43) CometColumnarExchange +(45) 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, CometColumnarShuffle, [plan_id=6] -(44) CometSort +(46) 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] -(45) ColumnarToRow [codegen id : 5] +(47) CometColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (72) -+- Exchange (71) - +- * HashAggregate (70) - +- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * ColumnarToRow (63) - : +- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (59) - : : +- CometBroadcastHashJoin (58) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (52) - : : +- CometBroadcastExchange (57) - : : +- CometFilter (56) - : : +- CometScan parquet spark_catalog.default.item (55) - : +- ReusedExchange (60) - +- ReusedExchange (64) - - -(46) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (76) ++- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * CometColumnarToRow (71) + +- CometColumnarExchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometProject (53) + : : : : +- CometSortMergeJoin (52) + : : : : :- CometSort (49) + : : : : : +- ReusedExchange (48) + : : : : +- CometSort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.item (57) + : +- ReusedExchange (62) + +- ReusedExchange (66) + + +(48) 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] -(47) CometSort +(49) 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_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] -(48) ReusedExchange [Reuses operator id: 9] +(50) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#47, sr_ticket_number#48] -(49) CometSort +(51) CometSort Input [2]: [sr_item_sk#47, sr_ticket_number#48] Arguments: [sr_item_sk#47, sr_ticket_number#48], [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST] -(50) CometSortMergeJoin +(52) 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 -(51) CometProject +(53) 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] -(52) ReusedExchange [Reuses operator id: 16] +(54) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] -(53) CometBroadcastHashJoin +(55) 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 -(54) CometProject +(56) 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] -(55) CometScan parquet spark_catalog.default.item +(57) CometScan 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 -(56) CometFilter +(58) 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) -(57) CometBroadcastExchange +(59) 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] -(58) CometBroadcastHashJoin +(60) 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 -(59) CometProject +(61) 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] -(60) ReusedExchange [Reuses operator id: 26] +(62) 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] -(61) CometBroadcastHashJoin +(63) 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 -(62) CometProject +(64) 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] -(63) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [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] -(64) ReusedExchange [Reuses operator id: 33] +(66) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(65) BroadcastHashJoin [codegen id : 2] +(67) 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 -(66) Project [codegen id : 2] +(68) 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] -(67) HashAggregate [codegen id : 2] +(69) 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] -(68) Exchange +(70) 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, CometColumnarShuffle, [plan_id=7] + +(71) CometColumnarToRow [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] -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=7] -(69) HashAggregate [codegen id : 3] +(72) 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] -(70) HashAggregate [codegen id : 3] +(73) 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] -(71) Exchange +(74) CometColumnarExchange +Input [2]: [sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(75) CometColumnarToRow [codegen id : 4] Input [2]: [sum#73, count#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(72) HashAggregate [codegen id : 4] +(76) 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 f4bcae0f13..27e6a9cd77 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 @@ -8,18 +8,55 @@ WholeStageCodegen (5) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + 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] + CometColumnarToRow 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 + 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 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 + CometColumnarToRow 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] @@ -30,62 +67,29 @@ WholeStageCodegen (5) 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 + CometExchange [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] - 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 + CometExchange [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] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + 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 - 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 (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] - 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 (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] - CometExchange [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] - CometExchange [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 #9 - WholeStageCodegen (1) - 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] + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + 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] 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 212c437e8e..bd83134312 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (69) +* CometColumnarToRow (69) +- CometTakeOrderedAndProject (68) +- CometUnion (67) :- CometHashAggregate (27) @@ -385,14 +385,14 @@ Child 2 Input [7]: [i_item_id#95, s_state#96, g_state#97, agg1#98, agg2#99, agg3 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] -(69) ColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [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 (74) -+- * ColumnarToRow (73) ++- * CometColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (74) BroadcastExchange 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 ed169acda8..8adbfa9ee8 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 2d2b384c87..25e9930cce 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 7da6076f5b..e1fc6bf43f 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 5933ea5fad..cc7702fe7b 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 0b55e23ab6..82c1f313df 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 c5bb669870..edc4b014bc 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometHashAggregate (36) +- CometExchange (35) @@ -215,14 +215,14 @@ Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_de 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] +(38) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 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] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (43) BroadcastExchange 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 5ebac30264..18c0a90bd8 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] 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 3dab08a149..8481ff7224 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,7 +2,7 @@ TakeOrderedAndProject (42) +- * Project (41) +- Window (40) - +- * ColumnarToRow (39) + +- * CometColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -220,7 +220,7 @@ Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, Co 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) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#40, i_category#11, i_class#10, lochierarchy#43, _w0#55] (40) Window @@ -239,7 +239,7 @@ Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (47) BroadcastExchange 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 6eb469dae6..9a3655e498 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [gross_margin,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 41c364a3f6..51357d654d 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [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] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_moy#11 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 a18f68d837..6355321c9b 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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-spark3_5/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt index 685f048dea..281d422ff4 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,82 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * ColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (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) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (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) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (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) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (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) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (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) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (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) CometScan parquet spark_catalog.default.web_sales @@ -172,7 +173,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] 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) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +274,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] 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) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +375,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] 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) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -406,50 +407,51 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) Exchange +(75) 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, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 11] +(76) 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#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(77) TakeOrderedAndProject +(77) 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] + +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: 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] ===== 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 (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) -(78) CometScan parquet spark_catalog.default.date_dim +(79) CometScan 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 +(80) 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] 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/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt index d98b2b0a66..21827c75d8 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,109 +1,110 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (11) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - 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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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 +WholeStageCodegen (11) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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 3211e46f6d..efa9fb0bdf 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,76 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Filter (71) - +- * HashAggregate (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- Window (61) - : +- * Sort (60) - : +- Exchange (59) - : +- * Project (58) - : +- * Filter (57) - : +- * SortMergeJoin FullOuter (56) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * ColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (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) - : +- * Sort (55) - : +- Exchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (42) - : : +- Window (41) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometExchange (36) - : : +- CometHashAggregate (35) - : : +- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (32) - : +- BroadcastExchange (48) - : +- * Project (47) - : +- Window (46) - : +- * ColumnarToRow (45) - : +- CometSort (44) - : +- ReusedExchange (43) - +- BroadcastExchange (66) - +- * Project (65) - +- Window (64) - +- * Sort (63) - +- ReusedExchange (62) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (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) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan parquet spark_catalog.default.web_sales @@ -135,7 +139,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (15) Window @@ -153,7 +157,7 @@ Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] 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) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] (20) Window @@ -185,26 +189,29 @@ 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) Exchange +(26) 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, [plan_id=4] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) HashAggregate [codegen id : 5] +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(28) 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] -(28) Exchange +(29) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) Sort [codegen id : 6] +(30) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(30) CometScan parquet spark_catalog.default.store_sales +(31) CometScan 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 [] @@ -212,222 +219,230 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Condition : isnotnull(ss_item_sk#23) -(32) ReusedExchange [Reuses operator id: 6] +(33) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#27, d_date#28] -(33) CometBroadcastHashJoin +(34) 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 -(34) CometProject +(35) 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] -(35) CometHashAggregate +(36) 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))] -(36) CometExchange +(37) CometExchange Input [3]: [ss_item_sk#23, d_date#28, sum#29] Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(37) CometHashAggregate +(38) 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))] -(38) CometExchange +(39) CometExchange Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort +(40) 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] -(40) ColumnarToRow [codegen id : 7] +(41) CometColumnarToRow [codegen id : 6] Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(41) Window +(42) 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] -(42) Project [codegen id : 10] +(43) 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] -(43) ReusedExchange [Reuses operator id: 38] +(44) ReusedExchange [Reuses operator id: 39] Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(44) CometSort +(45) 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] -(45) ColumnarToRow [codegen id : 8] +(46) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(46) Window +(47) 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] -(47) Project [codegen id : 9] +(48) 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] -(48) BroadcastExchange +(49) BroadcastExchange Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(49) BroadcastHashJoin [codegen id : 10] +(50) BroadcastHashJoin [codegen id : 9] Left keys [1]: [item_sk#30] Right keys [1]: [item_sk#36] Join type: Inner Join condition: (rk#32 >= rk#35) -(50) Project [codegen id : 10] +(51) 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] -(51) HashAggregate [codegen id : 10] +(52) 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] -(52) Exchange +(53) 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, [plan_id=9] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(53) HashAggregate [codegen id : 11] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) 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] -(54) Exchange +(56) CometColumnarExchange Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(55) Sort [codegen id : 12] +(57) CometSort Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(56) SortMergeJoin [codegen id : 13] -Left keys [2]: [item_sk#9, d_date#6] -Right keys [2]: [item_sk#30, d_date#28] -Join type: FullOuter -Join condition: None +(58) 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 -(57) Filter [codegen id : 13] +(59) 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) -(58) Project [codegen id : 13] -Output [4]: [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] +(60) 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] + +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(59) Exchange +(62) CometSort Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=11] +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] -(60) Sort [codegen id : 14] +(63) CometColumnarToRow [codegen id : 11] Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 -(61) Window +(64) 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] -(62) ReusedExchange [Reuses operator id: 59] +(65) ReusedExchange [Reuses operator id: 61] Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(63) Sort [codegen id : 28] +(66) CometSort Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 +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] -(64) Window +(67) CometColumnarToRow [codegen id : 22] +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#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(65) Project [codegen id : 29] +(69) 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] -(66) BroadcastExchange +(70) 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] -(67) BroadcastHashJoin [codegen id : 30] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [item_sk#44] Right keys [1]: [item_sk#50] Join type: Inner Join condition: (rk#48 >= rk#49) -(68) Project [codegen id : 30] +(72) 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] -(69) HashAggregate [codegen id : 30] +(73) 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] -(70) HashAggregate [codegen id : 30] +(74) 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] -(71) Filter [codegen id : 30] +(75) 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)) -(72) TakeOrderedAndProject +(76) 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 (77) -+- * ColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) -(73) CometScan parquet spark_catalog.default.date_dim +(77) CometScan 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 -(74) CometFilter +(78) 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)) -(75) CometProject +(79) 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] -(76) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(77) BroadcastExchange +(81) 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 = 30 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 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 f8e1ba2353..51540b97fd 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 (30) + 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,113 +7,110 @@ 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 (14) - Sort [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (13) - 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 (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #3 - 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 (1) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] - CometExchange [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 (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #9 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (10) - 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 (7) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] - CometExchange [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 (9) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + 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 (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometExchange [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) + CometColumnarToRow + 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 (3) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + 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 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + 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 (6) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometExchange [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 (8) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (29) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (28) - Sort [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow 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 4b3a3c576e..84c04cfce2 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 3ea3165fc7..e9ee530494 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index 3e1b88c26e..4cd6728185 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,93 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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) +* CometColumnarToRow (89) ++- CometTakeOrderedAndProject (88) + +- CometHashAggregate (87) + +- CometColumnarExchange (86) + +- * HashAggregate (85) + +- Union (84) + :- * HashAggregate (69) + : +- * CometColumnarToRow (68) + : +- CometColumnarExchange (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (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) + : : +- * CometColumnarToRow (40) + : : +- CometExchange (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) + : +- * CometColumnarToRow (63) + : +- CometExchange (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 (76) + : +- * CometColumnarToRow (75) + : +- CometColumnarExchange (74) + : +- * HashAggregate (73) + : +- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- ReusedExchange (70) + +- * HashAggregate (83) + +- * CometColumnarToRow (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- * HashAggregate (79) + +- * CometColumnarToRow (78) + +- ReusedExchange (77) (1) CometScan parquet spark_catalog.default.store_sales @@ -181,7 +187,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -272,7 +278,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled 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, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -382,7 +388,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -401,126 +407,142 @@ Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(prof 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) 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, [plan_id=4] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(68) HashAggregate [codegen id : 5] +(68) CometColumnarToRow [codegen id : 5] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(69) 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] +(70) ReusedExchange [Reuses operator id: 67] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(70) HashAggregate [codegen id : 10] +(71) CometColumnarToRow [codegen id : 10] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(72) 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 : 10] +(73) 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 +(74) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(73) HashAggregate [codegen id : 11] +(75) CometColumnarToRow [codegen id : 11] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(76) 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] +(77) ReusedExchange [Reuses operator id: 67] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(75) HashAggregate [codegen id : 16] +(78) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(79) 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 : 16] +(80) 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 +(81) CometColumnarExchange +Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(82) CometColumnarToRow [codegen id : 17] Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(78) HashAggregate [codegen id : 17] +(83) 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 +(84) Union -(80) HashAggregate [codegen id : 18] +(85) 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 +(86) 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, [plan_id=7] +Arguments: hashpartitioning(channel#34, id#35, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(82) HashAggregate [codegen id : 19] +(87) 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 +(88) 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] + +(89) CometColumnarToRow [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 (94) ++- * CometColumnarToRow (93) + +- CometProject (92) + +- CometFilter (91) + +- CometScan parquet spark_catalog.default.date_dim (90) -(84) CometScan parquet spark_catalog.default.date_dim +(90) CometScan 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 +(91) 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 +(92) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(94) 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 751b3dc240..712795e719 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,118 +1,124 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - 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)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometExchange [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 - CometExchange [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) - 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 - CometExchange [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] - InputAdapter - Exchange [channel] #12 - 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] - 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] - InputAdapter - Exchange #13 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + 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)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometExchange [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) + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #12 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #13 + 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] + CometColumnarToRow + 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 54cbf8326b..77ad43a121 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) 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] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [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 580f668ea8..266060bf09 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 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 4d8ac469c3..7252446de1 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC 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] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] 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 (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct 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)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct 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)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange 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 f38a6afca6..f4df8e5826 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 c866b93d2a..7cd8baf606 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,76 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * Filter (70) - +- Window (69) - +- WindowGroupLimit (68) - +- * Sort (67) - +- Exchange (66) - +- WindowGroupLimit (65) - +- * ColumnarToRow (64) - +- CometSort (63) - +- CometUnion (62) - :- CometHashAggregate (21) - : +- CometExchange (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) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- ReusedExchange (22) - :- CometHashAggregate (31) - : +- CometExchange (30) - : +- CometHashAggregate (29) - : +- CometHashAggregate (28) - : +- ReusedExchange (27) - :- CometHashAggregate (36) - : +- CometExchange (35) - : +- CometHashAggregate (34) - : +- CometHashAggregate (33) - : +- ReusedExchange (32) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometHashAggregate (38) - : +- ReusedExchange (37) - :- CometHashAggregate (46) - : +- CometExchange (45) - : +- CometHashAggregate (44) - : +- CometHashAggregate (43) - : +- ReusedExchange (42) - :- CometHashAggregate (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometHashAggregate (48) - : +- ReusedExchange (47) - :- CometHashAggregate (56) - : +- CometExchange (55) - : +- CometHashAggregate (54) - : +- CometHashAggregate (53) - : +- ReusedExchange (52) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometHashAggregate (58) - +- ReusedExchange (57) +TakeOrderedAndProject (72) ++- * Filter (71) + +- Window (70) + +- WindowGroupLimit (69) + +- * CometColumnarToRow (68) + +- CometSort (67) + +- CometColumnarExchange (66) + +- WindowGroupLimit (65) + +- * CometColumnarToRow (64) + +- CometSort (63) + +- CometUnion (62) + :- CometHashAggregate (21) + : +- CometExchange (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) + : +- CometExchange (25) + : +- CometHashAggregate (24) + : +- CometHashAggregate (23) + : +- ReusedExchange (22) + :- CometHashAggregate (31) + : +- CometExchange (30) + : +- CometHashAggregate (29) + : +- CometHashAggregate (28) + : +- ReusedExchange (27) + :- CometHashAggregate (36) + : +- CometExchange (35) + : +- CometHashAggregate (34) + : +- CometHashAggregate (33) + : +- ReusedExchange (32) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometHashAggregate (38) + : +- ReusedExchange (37) + :- CometHashAggregate (46) + : +- CometExchange (45) + : +- CometHashAggregate (44) + : +- CometHashAggregate (43) + : +- ReusedExchange (42) + :- CometHashAggregate (51) + : +- CometExchange (50) + : +- CometHashAggregate (49) + : +- CometHashAggregate (48) + : +- ReusedExchange (47) + :- CometHashAggregate (56) + : +- CometExchange (55) + : +- CometHashAggregate (54) + : +- CometHashAggregate (53) + : +- ReusedExchange (52) + +- CometHashAggregate (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometHashAggregate (58) + +- ReusedExchange (57) (1) CometScan parquet spark_catalog.default.store_sales @@ -365,66 +366,69 @@ Child 8 Input [9]: [i_category#177, i_class#178, i_brand#179, i_product_name#180 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] +(64) CometColumnarToRow [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] (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 -(66) Exchange +(66) 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, [plan_id=10] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(67) Sort [codegen id : 2] +(67) 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] + +(68) CometColumnarToRow [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] -Arguments: [i_category#17 ASC NULLS FIRST, sumsales#141 DESC NULLS LAST], false, 0 -(68) WindowGroupLimit +(69) 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 +(70) 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 : 3] +(71) 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 +(72) 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 (77) ++- * CometColumnarToRow (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(72) CometScan parquet spark_catalog.default.date_dim +(73) CometScan 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 +(74) 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 +(75) 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] +(76) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(76) BroadcastExchange +(77) 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 4ff3d4215c..1ee41a4774 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 @@ -5,81 +5,82 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ Window [sumsales,i_category] WindowGroupLimit [i_category,sumsales] WholeStageCodegen (2) - Sort [i_category,sumsales] + CometColumnarToRow InputAdapter - Exchange [i_category] #1 - 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))] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange [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)] - CometExchange #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 + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometColumnarExchange [i_category] #1 + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (1) + CometColumnarToRow + 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))] + CometExchange [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) + CometColumnarToRow + 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)] + CometExchange [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)] + CometExchange [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)] + CometExchange [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)] + CometExchange [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)] + CometExchange [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)] + CometExchange [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)] + CometExchange [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)] + CometExchange #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 d9cda55ca6..ed5d120dab 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,68 @@ == 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) - : +- * ColumnarToRow (27) - : +- CometSort (26) - : +- CometHashAggregate (25) - : +- CometExchange (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 (64) ++- * Project (63) + +- Window (62) + +- * CometColumnarToRow (61) + +- CometSort (60) + +- CometExchange (59) + +- CometHashAggregate (58) + +- CometColumnarExchange (57) + +- * HashAggregate (56) + +- Union (55) + :- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * CometColumnarToRow (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) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- WindowGroupLimit (28) + : +- * CometColumnarToRow (27) + : +- CometSort (26) + : +- CometHashAggregate (25) + : +- CometExchange (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 (47) + : +- * CometColumnarToRow (46) + : +- CometColumnarExchange (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- * CometColumnarToRow (42) + : +- ReusedExchange (41) + +- * HashAggregate (54) + +- * CometColumnarToRow (53) + +- CometColumnarExchange (52) + +- * HashAggregate (51) + +- * HashAggregate (50) + +- * CometColumnarToRow (49) + +- ReusedExchange (48) (1) CometScan parquet spark_catalog.default.store_sales @@ -99,7 +105,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 : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -113,7 +119,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 : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -182,7 +188,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] 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] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) WindowGroupLimit @@ -232,142 +238,158 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [3]: [s_state#9, s_county#8, sum#21] -(38) Exchange +(38) CometColumnarExchange Input [3]: [s_state#9, s_county#8, sum#21] -Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(39) HashAggregate [codegen id : 5] +(39) CometColumnarToRow [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))#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] -(40) ReusedExchange [Reuses operator id: 38] +(41) ReusedExchange [Reuses operator id: 38] Output [3]: [s_state#27, s_county#28, sum#29] -(41) HashAggregate [codegen id : 10] +(42) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#27, s_county#28, sum#29] + +(43) 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] -(42) HashAggregate [codegen id : 10] +(44) 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] -(43) Exchange +(45) CometColumnarExchange Input [3]: [s_state#27, sum#34, isEmpty#35] -Arguments: hashpartitioning(s_state#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(s_state#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(44) HashAggregate [codegen id : 11] +(46) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#27, sum#34, isEmpty#35] + +(47) 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] -(45) ReusedExchange [Reuses operator id: 38] +(48) ReusedExchange [Reuses operator id: 38] Output [3]: [s_state#42, s_county#43, sum#44] -(46) HashAggregate [codegen id : 16] +(49) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#42, s_county#43, sum#44] + +(50) 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] -(47) HashAggregate [codegen id : 16] +(51) HashAggregate [codegen id : 16] Input [1]: [total_sum#46] Keys: [] Functions [1]: [partial_sum(total_sum#46)] Aggregate Attributes [2]: [sum#47, isEmpty#48] Results [2]: [sum#49, isEmpty#50] -(48) Exchange +(52) CometColumnarExchange +Input [2]: [sum#49, isEmpty#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(53) CometColumnarToRow [codegen id : 17] Input [2]: [sum#49, isEmpty#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 17] +(54) HashAggregate [codegen id : 17] Input [2]: [sum#49, isEmpty#50] Keys: [] 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 +(55) Union -(51) HashAggregate [codegen id : 18] +(56) 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#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] -(52) Exchange +(57) 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, [plan_id=7] +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 : 19] +(58) 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#23, s_state#9, s_county#8, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#9 END AS _w0#58] -(54) Exchange +(59) CometExchange +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] +Arguments: hashpartitioning(lochierarchy#26, _w0#58, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(60) CometSort Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] -Arguments: hashpartitioning(lochierarchy#26, _w0#58, 5), ENSURE_REQUIREMENTS, [plan_id=8] +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 : 20] +(61) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#58] -Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#58 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0 -(56) Window +(62) 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 : 21] +(63) 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 +(64) 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 (69) ++- * CometColumnarToRow (68) + +- CometProject (67) + +- CometFilter (66) + +- CometScan parquet spark_catalog.default.date_dim (65) -(59) CometScan parquet spark_catalog.default.date_dim +(65) CometScan 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 +(66) 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 +(67) 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] +(68) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(63) BroadcastExchange +(69) 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 cb772b8002..b0a2be6feb 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,29 +1,29 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (21) + WholeStageCodegen (20) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (20) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (19) + CometColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (19) - 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 (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [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 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] + CometColumnarToRow InputAdapter - Exchange [s_state,s_county] #3 + CometColumnarExchange [s_state,s_county] #3 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 + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -58,7 +58,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] @@ -75,21 +75,25 @@ 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 (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow InputAdapter - Exchange [s_state] #10 + CometColumnarExchange [s_state] #10 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] - 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] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #11 + CometColumnarExchange #11 WholeStageCodegen (16) 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 + CometColumnarToRow + 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 45f7e2e66d..4b97b311be 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 12d5134a06..b1b1e31edb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange 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 631a82f1e3..9bde9ac863 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 6789a22468..b8bba79fa5 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[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]), [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], 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] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct 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)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct 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)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange 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 9e340130a5..5a8af58abc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 879652ae12..ee012cfa95 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,101 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (97) -+- * HashAggregate (96) - +- Exchange (95) - +- * HashAggregate (94) - +- Union (93) - :- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (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) - : : +- CometExchange (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) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * ColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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 (87) - : +- Exchange (86) - : +- * HashAggregate (85) - : +- * HashAggregate (84) - : +- ReusedExchange (83) - +- * HashAggregate (92) - +- Exchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- ReusedExchange (88) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * HashAggregate (83) + : +- * CometColumnarToRow (82) + : +- CometColumnarExchange (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometExchange (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) + : : +- CometExchange (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) + : : : +- * CometColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * CometColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * CometColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometExchange (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) + : +- CometExchange (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 (90) + : +- * CometColumnarToRow (89) + : +- CometColumnarExchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- * CometColumnarToRow (85) + : +- ReusedExchange (84) + +- * HashAggregate (97) + +- * CometColumnarToRow (96) + +- CometColumnarExchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- * CometColumnarToRow (92) + +- ReusedExchange (91) (1) CometScan parquet spark_catalog.default.store_sales @@ -239,7 +245,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight 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] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -275,7 +281,7 @@ 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))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -315,7 +321,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -451,7 +457,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight 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] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] (79) Union @@ -463,126 +469,142 @@ Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(prof 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) Exchange +(81) 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, [plan_id=8] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(82) HashAggregate [codegen id : 6] +(82) CometColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(83) 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] -(83) ReusedExchange [Reuses operator id: 81] +(84) ReusedExchange [Reuses operator id: 81] Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -(84) HashAggregate [codegen id : 12] +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) 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] -(85) HashAggregate [codegen id : 12] +(87) 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] -(86) Exchange +(88) CometColumnarExchange Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(87) HashAggregate [codegen id : 13] +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) 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] -(88) ReusedExchange [Reuses operator id: 81] +(91) ReusedExchange [Reuses operator id: 81] Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -(89) HashAggregate [codegen id : 19] +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(93) 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] -(90) HashAggregate [codegen id : 19] +(94) HashAggregate [codegen id : 19] Input [3]: [sales#116, returns#117, profit#118] Keys: [] 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] -(91) Exchange +(95) CometColumnarExchange +Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(96) CometColumnarToRow [codegen id : 20] Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(92) HashAggregate [codegen id : 20] +(97) HashAggregate [codegen id : 20] Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys: [] 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] -(93) Union +(98) Union -(94) HashAggregate [codegen id : 21] +(99) 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#24, id#25, sales#91, returns#92, profit#93] -(95) Exchange +(100) 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, [plan_id=11] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(96) HashAggregate [codegen id : 22] +(101) 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#24, id#25, sales#91, returns#92, profit#93] -(97) TakeOrderedAndProject +(102) 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] + +(103) CometColumnarToRow [codegen id : 22] Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [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 (102) -+- * ColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan parquet spark_catalog.default.date_dim (98) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(98) CometScan parquet spark_catalog.default.date_dim +(104) CometScan 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 -(99) CometFilter +(105) 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)) -(100) CometProject +(106) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(101) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(102) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] 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 01cd2f085f..02f19882bd 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,135 +1,141 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (22) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - 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))] - CometExchange [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))] - CometExchange [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] - 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 (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [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 +WholeStageCodegen (22) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (5) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #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 - 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))] - CometExchange [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 + 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))] + CometExchange [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) + CometColumnarToRow + 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))] + CometExchange [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] + 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 (3) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [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 + CometColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #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] - 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))] - CometExchange [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] - 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 (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] - InputAdapter - Exchange [channel] #16 - 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] - 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] - InputAdapter - Exchange #17 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (4) + CometColumnarToRow + 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))] + CometExchange [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))] + CometExchange [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] + 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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #16 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #17 + 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] + CometColumnarToRow + 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 1637692970..9a697cc844 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,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss 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] +(65) CometColumnarToRow [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] @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct 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)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange 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 78d5fe1d7b..58ca9f30b8 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,7 +1,7 @@ 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 (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] - ColumnarToRow + CometColumnarToRow InputAdapter 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] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 bd1ede55fb..64f76a0ce1 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,119 +1,125 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- Union (111) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * HashAggregate (38) - : : +- * ColumnarToRow (37) - : : +- CometExchange (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) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (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) - : : +- CometExchange (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) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (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) - : +- CometExchange (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) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (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 (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - +- * HashAggregate (110) - +- Exchange (109) - +- * HashAggregate (108) - +- * HashAggregate (107) - +- ReusedExchange (106) +* CometColumnarToRow (121) ++- CometTakeOrderedAndProject (120) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (101) + : +- * CometColumnarToRow (100) + : +- CometColumnarExchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * HashAggregate (38) + : : +- * CometColumnarToRow (37) + : : +- CometExchange (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) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (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) + : : +- * CometColumnarToRow (66) + : : +- CometExchange (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) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (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) + : +- * CometColumnarToRow (95) + : +- CometExchange (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) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (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 (108) + : +- * CometColumnarToRow (107) + : +- CometColumnarExchange (106) + : +- * HashAggregate (105) + : +- * HashAggregate (104) + : +- * CometColumnarToRow (103) + : +- ReusedExchange (102) + +- * HashAggregate (115) + +- * CometColumnarToRow (114) + +- CometColumnarExchange (113) + +- * HashAggregate (112) + +- * HashAggregate (111) + +- * CometColumnarToRow (110) + +- ReusedExchange (109) (1) CometScan parquet spark_catalog.default.store_sales @@ -285,7 +291,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -416,7 +422,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c 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, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [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] @@ -547,7 +553,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c 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, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -566,126 +572,142 @@ Functions [3]: [partial_sum(sales#32), partial_sum(returns#33), partial_sum(prof 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) Exchange +(99) 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, [plan_id=10] +Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(100) HashAggregate [codegen id : 5] +(100) CometColumnarToRow [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(101) 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] -(101) ReusedExchange [Reuses operator id: 99] +(102) ReusedExchange [Reuses operator id: 99] Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(102) HashAggregate [codegen id : 10] +(103) CometColumnarToRow [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(104) 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] -(103) HashAggregate [codegen id : 10] +(105) 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] -(104) Exchange +(106) CometColumnarExchange Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(105) HashAggregate [codegen id : 11] +(107) CometColumnarToRow [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(108) 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] -(106) ReusedExchange [Reuses operator id: 99] +(109) ReusedExchange [Reuses operator id: 99] Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(107) HashAggregate [codegen id : 16] +(110) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(111) 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] -(108) HashAggregate [codegen id : 16] +(112) HashAggregate [codegen id : 16] Input [3]: [sales#137, returns#138, profit#139] Keys: [] 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] -(109) Exchange +(113) CometColumnarExchange +Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(114) CometColumnarToRow [codegen id : 17] Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(110) HashAggregate [codegen id : 17] +(115) HashAggregate [codegen id : 17] Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] Keys: [] 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] -(111) Union +(116) Union -(112) HashAggregate [codegen id : 18] +(117) 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#30, id#31, sales#112, returns#113, profit#114] -(113) Exchange +(118) 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, [plan_id=13] +Arguments: hashpartitioning(channel#30, id#31, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(114) HashAggregate [codegen id : 19] +(119) 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#30, id#31, sales#112, returns#113, profit#114] -(115) TakeOrderedAndProject +(120) 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] + +(121) CometColumnarToRow [codegen id : 19] Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] -Arguments: 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [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 (120) -+- * ColumnarToRow (119) - +- CometProject (118) - +- CometFilter (117) - +- CometScan parquet spark_catalog.default.date_dim (116) +BroadcastExchange (126) ++- * CometColumnarToRow (125) + +- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(116) CometScan parquet spark_catalog.default.date_dim +(122) CometScan 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 -(117) CometFilter +(123) 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)) -(118) CometProject +(124) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(120) BroadcastExchange +(126) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] 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 aa61bbd579..582b39e19b 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,147 +1,153 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - 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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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] - InputAdapter - Exchange [channel] #19 - 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] - 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] - InputAdapter - Exchange #20 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #20 + 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] + CometColumnarToRow + 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 7354c23de0..6a35a64e6a 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,7 +2,7 @@ TakeOrderedAndProject (36) +- * Project (35) +- Window (34) - +- * ColumnarToRow (33) + +- * CometColumnarToRow (33) +- CometSort (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -186,7 +186,7 @@ Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, Co 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) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#25, i_category#9, i_class#8, lochierarchy#28, _w0#40] (34) Window @@ -205,7 +205,7 @@ Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -226,7 +226,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (41) BroadcastExchange 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 ae161e5ae2..d2ed714b7e 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [total_sum,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -24,7 +24,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 4d870a8b80..7bcf62d617 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,10 +1,10 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -111,7 +111,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -130,14 +130,14 @@ Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS 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] -(24) ColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [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 (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (29) BroadcastExchange 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 2b7813a707..6509498640 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,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -8,7 +8,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 @@ -24,7 +24,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 868b2f4822..c2b7f49205 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(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] 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] +(39) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange 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 2e8f2022cf..b570b700ac 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 6f50d7fbd9..d7b95a87ab 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,79 @@ == 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 (75) ++- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * CometColumnarToRow (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 (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.date_dim (26) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * HashAggregate (52) + : +- * CometColumnarToRow (51) + : +- CometColumnarExchange (50) + : +- * HashAggregate (49) + : +- * CometColumnarToRow (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- CometBroadcastExchange (42) + : : +- CometFilter (41) + : : +- CometScan parquet spark_catalog.default.web_sales (40) + : +- ReusedExchange (45) + +- BroadcastExchange (72) + +- * HashAggregate (71) + +- * CometColumnarToRow (70) + +- CometColumnarExchange (69) + +- * HashAggregate (68) + +- * CometColumnarToRow (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.customer (57) + : +- CometBroadcastExchange (61) + : +- CometFilter (60) + : +- CometScan parquet spark_catalog.default.web_sales (59) + +- ReusedExchange (64) (1) CometScan parquet spark_catalog.default.customer @@ -132,7 +136,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#14], Inner, BuildRight Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15], [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] (14) HashAggregate [codegen id : 1] @@ -142,33 +146,36 @@ 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) 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, [plan_id=1] +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] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarToRow [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] + +(17) 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] +(18) Filter [codegen id : 8] Input [2]: [customer_id#19, year_total#20] Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) -(18) CometScan parquet spark_catalog.default.customer +(19) CometScan 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 +(20) 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) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +183,93 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(22) 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 +(23) 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 +(24) 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 +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometBroadcastExchange Input [2]: [d_date_sk#34, d_year#35] Arguments: [d_date_sk#34, d_year#35] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) CometColumnarToRow [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] +(32) 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 +(33) 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] + +(34) CometColumnarToRow [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] +(35) 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 +(36) 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] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#19] Right keys [1]: [customer_id#38] Join type: Inner Join condition: None -(36) CometScan parquet spark_catalog.default.customer +(38) CometScan 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 +(39) 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) CometScan parquet spark_catalog.default.web_sales +(40) CometScan 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 +277,89 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(41) 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 +(42) 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 +(43) 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#56, d_year#57] -(44) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) CometColumnarToRow [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] +(49) 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 +(50) 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, [plan_id=4] +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] -(49) HashAggregate [codegen id : 5] +(51) CometColumnarToRow [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] + +(52) 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] +(53) 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 +(54) 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] +(55) 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] +(56) 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) CometScan parquet spark_catalog.default.customer +(57) CometScan 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 +(58) 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) CometScan parquet spark_catalog.default.web_sales +(59) CometScan 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 +367,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(60) 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 +(61) 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 +(62) 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 +(63) 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] +(64) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#76, d_year#77] -(62) CometBroadcastHashJoin +(65) 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 +(66) 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] +(67) CometColumnarToRow [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] +(68) 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 +(69) 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] + +(70) CometColumnarToRow [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] +(71) 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 +(72) 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] +(73) 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] +(74) 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 +(75) 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 (79) ++- * CometColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(72) CometScan parquet spark_catalog.default.date_dim +(76) CometScan 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 +(77) 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] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] -(75) BroadcastExchange +(79) 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 = 21 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(76) CometScan parquet spark_catalog.default.date_dim +(80) CometScan 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#34, d_year#35] -(79) BroadcastExchange +(83) 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 = 40 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 = 59 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 d7c6ef3b25..6c107201a7 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,100 +7,104 @@ 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 96b5da2358..44b8a695df 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,28 @@ == 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 (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.web_sales @@ -87,7 +89,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -97,66 +99,72 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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 +(24) 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) CometScan parquet spark_catalog.default.date_dim +(25) CometScan 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 +(26) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject +(27) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=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 07c91b94dc..de0b48fe9d 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,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 143f2c1b57..cde256d6d1 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,90 +1,92 @@ == Physical Plan == -TakeOrderedAndProject (86) -+- * BroadcastHashJoin Inner BuildRight (85) - :- * Filter (65) - : +- * HashAggregate (64) - : +- Exchange (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) - : : : : +- CometExchange (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 (84) - +- * Filter (83) - +- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- * ColumnarToRow (79) - +- CometProject (78) - +- CometBroadcastHashJoin (77) - :- CometProject (72) - : +- CometBroadcastHashJoin (71) - : :- CometBroadcastHashJoin (69) - : : :- CometFilter (67) - : : : +- CometScan parquet spark_catalog.default.store_sales (66) - : : +- ReusedExchange (68) - : +- ReusedExchange (70) - +- CometBroadcastExchange (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +TakeOrderedAndProject (88) ++- * BroadcastHashJoin Inner BuildRight (87) + :- * Filter (66) + : +- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (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) + : : : : +- CometExchange (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 (86) + +- * Filter (85) + +- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * CometColumnarToRow (80) + +- CometProject (79) + +- CometBroadcastHashJoin (78) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (70) + : : :- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (69) + : +- ReusedExchange (71) + +- CometBroadcastExchange (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) (1) CometScan parquet spark_catalog.default.store_sales @@ -372,7 +374,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight 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] +(61) CometColumnarToRow [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] @@ -382,22 +384,25 @@ Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_pric 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] -(63) Exchange +(63) 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, [plan_id=2] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(64) HashAggregate [codegen id : 4] +(64) CometColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#50, isEmpty#51, count#52] + +(65) 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))#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] -(65) Filter [codegen id : 4] +(66) 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)))) -(66) CometScan parquet spark_catalog.default.store_sales +(67) CometScan 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 [] @@ -405,292 +410,299 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(67) CometFilter +(68) 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) -(68) ReusedExchange [Reuses operator id: 46] +(69) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#65] -(69) CometBroadcastHashJoin +(70) 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 -(70) ReusedExchange [Reuses operator id: 52] +(71) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69] -(71) CometBroadcastHashJoin +(72) 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 -(72) CometProject +(73) 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] -(73) CometScan parquet spark_catalog.default.date_dim +(74) CometScan 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 -(74) CometFilter +(75) 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)) -(75) CometProject +(76) CometProject Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(76) CometBroadcastExchange +(77) CometBroadcastExchange Input [1]: [d_date_sk#70] Arguments: [d_date_sk#70] -(77) CometBroadcastHashJoin +(78) 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 -(78) CometProject +(79) 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] -(79) ColumnarToRow [codegen id : 2] +(80) CometColumnarToRow [codegen id : 2] Input [5]: [ss_quantity#61, ss_list_price#62, i_brand_id#67, i_class_id#68, i_category_id#69] -(80) HashAggregate [codegen id : 2] +(81) 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] -(81) Exchange +(82) 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] + +(83) CometColumnarToRow [codegen id : 3] 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, [plan_id=3] -(82) HashAggregate [codegen id : 3] +(84) 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] -(83) Filter [codegen id : 3] +(85) 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)))) -(84) BroadcastExchange +(86) 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] -(85) BroadcastHashJoin [codegen id : 4] +(87) 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#67, i_class_id#68, i_category_id#69] Join type: Inner Join condition: None -(86) TakeOrderedAndProject +(88) 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 = 65 Hosting Expression = Subquery scalar-subquery#58, [id=#59] -* HashAggregate (103) -+- Exchange (102) - +- * HashAggregate (101) - +- * ColumnarToRow (100) - +- CometUnion (99) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometScan parquet spark_catalog.default.store_sales (87) - : +- ReusedExchange (88) - :- CometProject (94) - : +- CometBroadcastHashJoin (93) - : :- CometScan parquet spark_catalog.default.catalog_sales (91) - : +- ReusedExchange (92) - +- CometProject (98) - +- CometBroadcastHashJoin (97) - :- CometScan parquet spark_catalog.default.web_sales (95) - +- ReusedExchange (96) - - -(87) CometScan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#58, [id=#59] +* HashAggregate (106) ++- * CometColumnarToRow (105) + +- CometColumnarExchange (104) + +- * HashAggregate (103) + +- * CometColumnarToRow (102) + +- CometUnion (101) + :- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometScan parquet spark_catalog.default.store_sales (89) + : +- ReusedExchange (90) + :- CometProject (96) + : +- CometBroadcastHashJoin (95) + : :- CometScan parquet spark_catalog.default.catalog_sales (93) + : +- ReusedExchange (94) + +- CometProject (100) + +- CometBroadcastHashJoin (99) + :- CometScan parquet spark_catalog.default.web_sales (97) + +- ReusedExchange (98) + + +(89) CometScan 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#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] ReadSchema: struct -(88) ReusedExchange [Reuses operator id: 19] +(90) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#89] -(89) CometBroadcastHashJoin +(91) 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 -(90) CometProject +(92) 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] -(91) CometScan parquet spark_catalog.default.catalog_sales +(93) CometScan 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#94), dynamicpruningexpression(cs_sold_date_sk#94 IN dynamicpruning#95)] ReadSchema: struct -(92) ReusedExchange [Reuses operator id: 19] +(94) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#96] -(93) CometBroadcastHashJoin +(95) 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 -(94) CometProject +(96) 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] -(95) CometScan parquet spark_catalog.default.web_sales +(97) CometScan 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#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#102)] ReadSchema: struct -(96) ReusedExchange [Reuses operator id: 19] +(98) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#103] -(97) CometBroadcastHashJoin +(99) 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 -(98) CometProject +(100) 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] -(99) CometUnion +(101) 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] -(100) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#90, list_price#91] -(101) HashAggregate [codegen id : 1] +(103) HashAggregate [codegen id : 1] Input [2]: [quantity#90, list_price#91] Keys: [] 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] -(102) Exchange +(104) CometColumnarExchange +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(105) CometColumnarToRow [codegen id : 2] Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(103) HashAggregate [codegen id : 2] +(106) HashAggregate [codegen id : 2] Input [2]: [sum#108, count#109] Keys: [] 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 = 87 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 89 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 +Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#94 IN dynamicpruning#12 -Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#12 +Subquery:4 Hosting operator id = 97 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 (108) -+- * ColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan parquet spark_catalog.default.date_dim (104) +BroadcastExchange (111) ++- * CometColumnarToRow (110) + +- CometProject (109) + +- CometFilter (108) + +- CometScan parquet spark_catalog.default.date_dim (107) -(104) CometScan parquet spark_catalog.default.date_dim +(107) CometScan 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 -(105) CometFilter +(108) 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)) -(106) CometProject +(109) CometProject Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(107) ColumnarToRow [codegen id : 1] +(110) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(108) BroadcastExchange +(111) BroadcastExchange Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] +Subquery:6 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (112) -+- CometProject (111) - +- CometFilter (110) - +- CometScan parquet spark_catalog.default.date_dim (109) +Subquery:7 Hosting operator id = 107 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +* CometColumnarToRow (115) ++- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(109) CometScan parquet spark_catalog.default.date_dim +(112) CometScan 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 -(110) CometFilter +(113) 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)) -(111) CometProject +(114) 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] -(112) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#112] Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (117) -+- * ColumnarToRow (116) - +- CometProject (115) - +- CometFilter (114) - +- CometScan parquet spark_catalog.default.date_dim (113) +BroadcastExchange (120) ++- * CometColumnarToRow (119) + +- CometProject (118) + +- CometFilter (117) + +- CometScan parquet spark_catalog.default.date_dim (116) -(113) CometScan parquet spark_catalog.default.date_dim +(116) CometScan 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 -(114) CometFilter +(117) 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)) -(115) CometProject +(118) CometProject Input [2]: [d_date_sk#26, d_year#116] Arguments: [d_date_sk#26], [d_date_sk#26] -(116) ColumnarToRow [codegen id : 1] +(119) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(117) BroadcastExchange +(120) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] @@ -702,67 +714,67 @@ Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] -Subquery:13 Hosting operator id = 83 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] +Subquery:13 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#58, [id=#59] -Subquery:14 Hosting operator id = 66 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (122) -+- * ColumnarToRow (121) - +- CometProject (120) - +- CometFilter (119) - +- CometScan parquet spark_catalog.default.date_dim (118) +Subquery:14 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 +BroadcastExchange (125) ++- * CometColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) -(118) CometScan parquet spark_catalog.default.date_dim +(121) CometScan 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 -(119) CometFilter +(122) 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)) -(120) CometProject +(123) CometProject Input [2]: [d_date_sk#70, d_week_seq#71] Arguments: [d_date_sk#70], [d_date_sk#70] -(121) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#70] -(122) BroadcastExchange +(125) BroadcastExchange Input [1]: [d_date_sk#70] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:15 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:15 Hosting operator id = 122 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:16 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#72, [id=#73] -* ColumnarToRow (126) -+- CometProject (125) - +- CometFilter (124) - +- CometScan parquet spark_catalog.default.date_dim (123) +Subquery:16 Hosting operator id = 121 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +* CometColumnarToRow (129) ++- CometProject (128) + +- CometFilter (127) + +- CometScan parquet spark_catalog.default.date_dim (126) -(123) CometScan parquet spark_catalog.default.date_dim +(126) CometScan parquet spark_catalog.default.date_dim Output [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct -(124) CometFilter +(127) CometFilter Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Condition : (((((isnotnull(d_year#118) AND isnotnull(d_moy#119)) AND isnotnull(d_dom#120)) AND (d_year#118 = 1998)) AND (d_moy#119 = 12)) AND (d_dom#120 = 16)) -(125) CometProject +(128) CometProject Input [4]: [d_week_seq#117, d_year#118, d_moy#119, d_dom#120] Arguments: [d_week_seq#117], [d_week_seq#117] -(126) ColumnarToRow [codegen id : 1] +(129) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#117] -Subquery:17 Hosting operator id = 74 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:17 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:18 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:18 Hosting operator id = 74 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 1fecc0fca4..5ac344b17d 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 @@ -5,163 +5,166 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Subquery #4 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] #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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #14 + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + CometColumnarToRow + 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 (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 #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] - 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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (1) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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] + 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] + CometExchange [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) + CometColumnarToRow + 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] - 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] + 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] + ReusedSubquery [d_week_seq] #2 InputAdapter 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] #16 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 + WholeStageCodegen (2) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + 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 14e2e96b2e..3e54b5ffdd 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,128 +1,141 @@ == Physical Plan == -TakeOrderedAndProject (124) -+- * HashAggregate (123) - +- Exchange (122) - +- * HashAggregate (121) - +- Union (120) - :- * HashAggregate (99) - : +- Exchange (98) - : +- * HashAggregate (97) - : +- Union (96) - : :- * Filter (65) - : : +- * HashAggregate (64) - : : +- Exchange (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) - : : : : : +- CometExchange (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 (80) - : : +- * HashAggregate (79) - : : +- Exchange (78) - : : +- * HashAggregate (77) - : : +- * ColumnarToRow (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) - : +- * Filter (95) - : +- * HashAggregate (94) - : +- Exchange (93) - : +- * HashAggregate (92) - : +- * ColumnarToRow (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) - :- * HashAggregate (104) - : +- Exchange (103) - : +- * HashAggregate (102) - : +- * HashAggregate (101) - : +- ReusedExchange (100) - :- * HashAggregate (109) - : +- Exchange (108) - : +- * HashAggregate (107) - : +- * HashAggregate (106) - : +- ReusedExchange (105) - :- * HashAggregate (114) - : +- Exchange (113) - : +- * HashAggregate (112) - : +- * HashAggregate (111) - : +- ReusedExchange (110) - +- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- * HashAggregate (116) - +- ReusedExchange (115) +* CometColumnarToRow (137) ++- CometTakeOrderedAndProject (136) + +- CometHashAggregate (135) + +- CometColumnarExchange (134) + +- * HashAggregate (133) + +- Union (132) + :- * HashAggregate (103) + : +- * CometColumnarToRow (102) + : +- CometColumnarExchange (101) + : +- * HashAggregate (100) + : +- Union (99) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- * CometColumnarToRow (64) + : : +- CometColumnarExchange (63) + : : +- * HashAggregate (62) + : : +- * CometColumnarToRow (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) + : : : : : +- CometExchange (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 (82) + : : +- * HashAggregate (81) + : : +- * CometColumnarToRow (80) + : : +- CometColumnarExchange (79) + : : +- * HashAggregate (78) + : : +- * CometColumnarToRow (77) + : : +- CometProject (76) + : : +- CometBroadcastHashJoin (75) + : : :- CometProject (73) + : : : +- CometBroadcastHashJoin (72) + : : : :- CometBroadcastHashJoin (70) + : : : : :- CometFilter (68) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) + : : : : +- ReusedExchange (69) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (74) + : +- * Filter (98) + : +- * HashAggregate (97) + : +- * CometColumnarToRow (96) + : +- CometColumnarExchange (95) + : +- * HashAggregate (94) + : +- * CometColumnarToRow (93) + : +- CometProject (92) + : +- CometBroadcastHashJoin (91) + : :- CometProject (89) + : : +- CometBroadcastHashJoin (88) + : : :- CometBroadcastHashJoin (86) + : : : :- CometFilter (84) + : : : : +- CometScan parquet spark_catalog.default.web_sales (83) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (110) + : +- * CometColumnarToRow (109) + : +- CometColumnarExchange (108) + : +- * HashAggregate (107) + : +- * HashAggregate (106) + : +- * CometColumnarToRow (105) + : +- ReusedExchange (104) + :- * HashAggregate (117) + : +- * CometColumnarToRow (116) + : +- CometColumnarExchange (115) + : +- * HashAggregate (114) + : +- * HashAggregate (113) + : +- * CometColumnarToRow (112) + : +- ReusedExchange (111) + :- * HashAggregate (124) + : +- * CometColumnarToRow (123) + : +- CometColumnarExchange (122) + : +- * HashAggregate (121) + : +- * HashAggregate (120) + : +- * CometColumnarToRow (119) + : +- ReusedExchange (118) + +- * HashAggregate (131) + +- * CometColumnarToRow (130) + +- CometColumnarExchange (129) + +- * HashAggregate (128) + +- * HashAggregate (127) + +- * CometColumnarToRow (126) + +- ReusedExchange (125) (1) CometScan parquet spark_catalog.default.store_sales @@ -410,7 +423,7 @@ Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight 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] +(61) CometColumnarToRow [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] @@ -420,22 +433,25 @@ Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_pric 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) Exchange +(63) 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, [plan_id=2] +Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(64) HashAggregate [codegen id : 2] +(64) CometColumnarToRow [codegen id : 2] +Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#49, isEmpty#50, count#51] + +(65) 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))#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] -(65) Filter [codegen id : 2] +(66) 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)))) -(66) CometScan parquet spark_catalog.default.catalog_sales +(67) CometScan 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 [] @@ -443,68 +459,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(67) CometFilter +(68) 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) -(68) ReusedExchange [Reuses operator id: 46] +(69) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#64] -(69) CometBroadcastHashJoin +(70) 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 -(70) ReusedExchange [Reuses operator id: 52] +(71) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] -(71) CometBroadcastHashJoin +(72) 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 -(72) CometProject +(73) 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] -(73) ReusedExchange [Reuses operator id: 58] +(74) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#69] -(74) CometBroadcastHashJoin +(75) 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 -(75) CometProject +(76) 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] -(76) ColumnarToRow [codegen id : 3] +(77) CometColumnarToRow [codegen id : 3] Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] -(77) HashAggregate [codegen id : 3] +(78) 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] -(78) Exchange +(79) 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, [plan_id=3] +Arguments: hashpartitioning(i_brand_id#66, i_class_id#67, i_category_id#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(79) HashAggregate [codegen id : 4] +(80) CometColumnarToRow [codegen id : 4] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] + +(81) 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] -(80) Filter [codegen id : 4] +(82) 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)))) -(81) CometScan parquet spark_catalog.default.web_sales +(83) CometScan 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 [] @@ -512,438 +531,473 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(82) CometFilter +(84) 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) -(83) ReusedExchange [Reuses operator id: 46] +(85) ReusedExchange [Reuses operator id: 46] Output [1]: [ss_item_sk#86] -(84) CometBroadcastHashJoin +(86) 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 -(85) ReusedExchange [Reuses operator id: 52] +(87) ReusedExchange [Reuses operator id: 52] Output [4]: [i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] -(86) CometBroadcastHashJoin +(88) 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 -(87) CometProject +(89) 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] -(88) ReusedExchange [Reuses operator id: 58] +(90) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date_sk#91] -(89) CometBroadcastHashJoin +(91) 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 -(90) CometProject +(92) 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] -(91) ColumnarToRow [codegen id : 5] +(93) CometColumnarToRow [codegen id : 5] Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] -(92) HashAggregate [codegen id : 5] +(94) 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] -(93) Exchange +(95) 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] + +(96) CometColumnarToRow [codegen id : 6] Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] -Arguments: hashpartitioning(i_brand_id#88, i_class_id#89, i_category_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(94) HashAggregate [codegen id : 6] +(97) 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] -(95) Filter [codegen id : 6] +(98) 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)))) -(96) Union +(99) Union -(97) HashAggregate [codegen id : 7] +(100) 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] -(98) Exchange +(101) 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, [plan_id=5] +Arguments: hashpartitioning(channel#54, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(99) HashAggregate [codegen id : 8] +(102) CometColumnarToRow [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] + +(103) 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] -(100) ReusedExchange [Reuses operator id: 98] +(104) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum#117, isEmpty#118, sum#119] -(101) HashAggregate [codegen id : 16] +(105) CometColumnarToRow [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] + +(106) 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] -(102) HashAggregate [codegen id : 16] +(107) 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] -(103) Exchange +(108) 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] + +(109) CometColumnarToRow [codegen id : 17] 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, [plan_id=6] -(104) HashAggregate [codegen id : 17] +(110) 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] -(105) ReusedExchange [Reuses operator id: 98] +(111) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#135, i_brand_id#136, i_class_id#137, i_category_id#138, sum#139, isEmpty#140, sum#141] -(106) HashAggregate [codegen id : 25] +(112) CometColumnarToRow [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] + +(113) 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] -(107) HashAggregate [codegen id : 25] +(114) 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] -(108) Exchange +(115) 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] + +(116) CometColumnarToRow [codegen id : 26] 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, [plan_id=7] -(109) HashAggregate [codegen id : 26] +(117) 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] -(110) ReusedExchange [Reuses operator id: 98] +(118) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#158, i_brand_id#159, i_class_id#160, i_category_id#161, sum#162, isEmpty#163, sum#164] -(111) HashAggregate [codegen id : 34] +(119) CometColumnarToRow [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] + +(120) 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] -(112) HashAggregate [codegen id : 34] +(121) 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] -(113) Exchange +(122) CometColumnarExchange +Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] +Arguments: hashpartitioning(channel#158, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(123) CometColumnarToRow [codegen id : 35] Input [4]: [channel#158, sum#172, isEmpty#173, sum#174] -Arguments: hashpartitioning(channel#158, 5), ENSURE_REQUIREMENTS, [plan_id=8] -(114) HashAggregate [codegen id : 35] +(124) 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] -(115) ReusedExchange [Reuses operator id: 98] +(125) ReusedExchange [Reuses operator id: 101] Output [7]: [channel#182, i_brand_id#183, i_class_id#184, i_category_id#185, sum#186, isEmpty#187, sum#188] -(116) HashAggregate [codegen id : 43] +(126) CometColumnarToRow [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] + +(127) 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] -(117) HashAggregate [codegen id : 43] +(128) HashAggregate [codegen id : 43] Input [2]: [sum_sales#191, number_sales#192] Keys: [] 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] -(118) Exchange +(129) CometColumnarExchange +Input [3]: [sum#196, isEmpty#197, sum#198] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(130) CometColumnarToRow [codegen id : 44] Input [3]: [sum#196, isEmpty#197, sum#198] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] -(119) HashAggregate [codegen id : 44] +(131) HashAggregate [codegen id : 44] Input [3]: [sum#196, isEmpty#197, sum#198] Keys: [] 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] -(120) Union +(132) Union -(121) HashAggregate [codegen id : 45] +(133) 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#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] -(122) Exchange +(134) 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, [plan_id=10] +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] -(123) HashAggregate [codegen id : 46] +(135) 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#54, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#111, number_sales#112] -(124) TakeOrderedAndProject +(136) 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] + +(137) CometColumnarToRow [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] -Arguments: 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] ===== Subqueries ===== -Subquery:1 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#57, [id=#58] -* HashAggregate (144) -+- Exchange (143) - +- * HashAggregate (142) - +- * ColumnarToRow (141) - +- CometUnion (140) - :- CometProject (128) - : +- CometBroadcastHashJoin (127) - : :- CometScan parquet spark_catalog.default.store_sales (125) - : +- ReusedExchange (126) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometScan parquet spark_catalog.default.catalog_sales (129) - : +- CometBroadcastExchange (133) - : +- CometProject (132) - : +- CometFilter (131) - : +- CometScan parquet spark_catalog.default.date_dim (130) - +- CometProject (139) - +- CometBroadcastHashJoin (138) - :- CometScan parquet spark_catalog.default.web_sales (136) - +- ReusedExchange (137) - - -(125) CometScan parquet spark_catalog.default.store_sales +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#57, [id=#58] +* HashAggregate (158) ++- * CometColumnarToRow (157) + +- CometColumnarExchange (156) + +- * HashAggregate (155) + +- * CometColumnarToRow (154) + +- CometUnion (153) + :- CometProject (141) + : +- CometBroadcastHashJoin (140) + : :- CometScan parquet spark_catalog.default.store_sales (138) + : +- ReusedExchange (139) + :- CometProject (148) + : +- CometBroadcastHashJoin (147) + : :- CometScan parquet spark_catalog.default.catalog_sales (142) + : +- CometBroadcastExchange (146) + : +- CometProject (145) + : +- CometFilter (144) + : +- CometScan parquet spark_catalog.default.date_dim (143) + +- CometProject (152) + +- CometBroadcastHashJoin (151) + :- CometScan parquet spark_catalog.default.web_sales (149) + +- ReusedExchange (150) + + +(138) CometScan 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#209), dynamicpruningexpression(ss_sold_date_sk#209 IN dynamicpruning#210)] ReadSchema: struct -(126) ReusedExchange [Reuses operator id: 19] +(139) ReusedExchange [Reuses operator id: 19] Output [1]: [d_date_sk#211] -(127) CometBroadcastHashJoin +(140) 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 -(128) CometProject +(141) 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] -(129) CometScan parquet spark_catalog.default.catalog_sales +(142) CometScan 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#216), dynamicpruningexpression(cs_sold_date_sk#216 IN dynamicpruning#217)] ReadSchema: struct -(130) CometScan parquet spark_catalog.default.date_dim +(143) CometScan 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 -(131) CometFilter +(144) 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)) -(132) CometProject +(145) CometProject Input [2]: [d_date_sk#218, d_year#219] Arguments: [d_date_sk#218], [d_date_sk#218] -(133) CometBroadcastExchange +(146) CometBroadcastExchange Input [1]: [d_date_sk#218] Arguments: [d_date_sk#218] -(134) CometBroadcastHashJoin +(147) 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 -(135) CometProject +(148) 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] -(136) CometScan parquet spark_catalog.default.web_sales +(149) CometScan 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#224), dynamicpruningexpression(ws_sold_date_sk#224 IN dynamicpruning#225)] ReadSchema: struct -(137) ReusedExchange [Reuses operator id: 133] +(150) ReusedExchange [Reuses operator id: 146] Output [1]: [d_date_sk#226] -(138) CometBroadcastHashJoin +(151) 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 -(139) CometProject +(152) 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] -(140) CometUnion +(153) 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] -(141) ColumnarToRow [codegen id : 1] +(154) CometColumnarToRow [codegen id : 1] Input [2]: [quantity#212, list_price#213] -(142) HashAggregate [codegen id : 1] +(155) HashAggregate [codegen id : 1] Input [2]: [quantity#212, list_price#213] Keys: [] 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] -(143) Exchange +(156) CometColumnarExchange +Input [2]: [sum#231, count#232] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] + +(157) CometColumnarToRow [codegen id : 2] Input [2]: [sum#231, count#232] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] -(144) HashAggregate [codegen id : 2] +(158) HashAggregate [codegen id : 2] Input [2]: [sum#231, count#232] Keys: [] 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 = 125 Hosting Expression = ss_sold_date_sk#209 IN dynamicpruning#12 +Subquery:2 Hosting operator id = 138 Hosting Expression = ss_sold_date_sk#209 IN dynamicpruning#12 -Subquery:3 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#216 IN dynamicpruning#217 -BroadcastExchange (149) -+- * ColumnarToRow (148) - +- CometProject (147) - +- CometFilter (146) - +- CometScan parquet spark_catalog.default.date_dim (145) +Subquery:3 Hosting operator id = 142 Hosting Expression = cs_sold_date_sk#216 IN dynamicpruning#217 +BroadcastExchange (163) ++- * CometColumnarToRow (162) + +- CometProject (161) + +- CometFilter (160) + +- CometScan parquet spark_catalog.default.date_dim (159) -(145) CometScan parquet spark_catalog.default.date_dim +(159) CometScan 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 -(146) CometFilter +(160) 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)) -(147) CometProject +(161) CometProject Input [2]: [d_date_sk#218, d_year#219] Arguments: [d_date_sk#218], [d_date_sk#218] -(148) ColumnarToRow [codegen id : 1] +(162) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#218] -(149) BroadcastExchange +(163) 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 = 136 Hosting Expression = ws_sold_date_sk#224 IN dynamicpruning#217 +Subquery:4 Hosting operator id = 149 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 (154) -+- * ColumnarToRow (153) - +- CometProject (152) - +- CometFilter (151) - +- CometScan parquet spark_catalog.default.date_dim (150) +BroadcastExchange (168) ++- * CometColumnarToRow (167) + +- CometProject (166) + +- CometFilter (165) + +- CometScan parquet spark_catalog.default.date_dim (164) -(150) CometScan parquet spark_catalog.default.date_dim +(164) CometScan 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 -(151) CometFilter +(165) 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)) -(152) CometProject +(166) CometProject Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(153) ColumnarToRow [codegen id : 1] +(167) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] -(154) BroadcastExchange +(168) BroadcastExchange Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (159) -+- * ColumnarToRow (158) - +- CometProject (157) - +- CometFilter (156) - +- CometScan parquet spark_catalog.default.date_dim (155) +BroadcastExchange (173) ++- * CometColumnarToRow (172) + +- CometProject (171) + +- CometFilter (170) + +- CometScan parquet spark_catalog.default.date_dim (169) -(155) CometScan parquet spark_catalog.default.date_dim +(169) CometScan parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#26, d_year#235] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct -(156) CometFilter +(170) CometFilter Input [2]: [d_date_sk#26, d_year#235] Condition : (((isnotnull(d_year#235) AND (d_year#235 >= 1999)) AND (d_year#235 <= 2001)) AND isnotnull(d_date_sk#26)) -(157) CometProject +(171) CometProject Input [2]: [d_date_sk#26, d_year#235] Arguments: [d_date_sk#26], [d_date_sk#26] -(158) ColumnarToRow [codegen id : 1] +(172) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] -(159) BroadcastExchange +(173) BroadcastExchange Input [1]: [d_date_sk#26] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] @@ -951,12 +1005,12 @@ Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dy Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 80 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 95 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#57, [id=#58] -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 83 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 1610cd8884..a8fe3eb75c 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,217 +1,231 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (46) - 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 (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] - InputAdapter - Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - 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] - InputAdapter - Exchange #16 - 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] - 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] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #3 - 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] - CometExchange [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] - 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] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #19 - WholeStageCodegen (3) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - ColumnarToRow +WholeStageCodegen (46) + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 + 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] + CometColumnarToRow 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] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #20 - 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] - InputAdapter - Exchange [channel,i_brand_id,i_class_id] #21 - 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] - 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] - InputAdapter - Exchange [channel,i_brand_id] #22 - 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] - 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] - InputAdapter - Exchange [channel] #23 - 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] - 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] - InputAdapter - Exchange #24 - 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] - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 + CometColumnarExchange #16 + WholeStageCodegen (1) + HashAggregate [quantity,list_price] [sum,count,sum,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (1) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #19 + WholeStageCodegen (3) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #20 + WholeStageCodegen (5) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id,i_class_id] #21 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,i_brand_id] #22 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #23 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #24 + 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] + CometColumnarToRow + 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 9aea34f5ef..4f061b8eb1 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,156 @@ == 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 (152) ++- Union (151) + :- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- CometColumnarExchange (38) + : +- * HashAggregate (37) + : +- * CometColumnarToRow (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 (67) + : +- * CometColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- * HashAggregate (64) + : +- * CometColumnarToRow (63) + : +- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometProject (59) + : : +- CometBroadcastHashJoin (58) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometProject (51) + : : : : +- CometBroadcastHashJoin (50) + : : : : :- CometProject (48) + : : : : : +- CometBroadcastHashJoin (47) + : : : : : :- CometProject (45) + : : : : : : +- CometBroadcastHashJoin (44) + : : : : : : :- CometFilter (42) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (41) + : : : : : : +- ReusedExchange (43) + : : : : : +- ReusedExchange (46) + : : : : +- ReusedExchange (49) + : : : +- CometBroadcastExchange (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.customer_address (52) + : : +- ReusedExchange (57) + : +- ReusedExchange (60) + :- * HashAggregate (95) + : +- * CometColumnarToRow (94) + : +- CometColumnarExchange (93) + : +- * HashAggregate (92) + : +- * CometColumnarToRow (91) + : +- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometProject (87) + : : +- CometBroadcastHashJoin (86) + : : :- CometProject (84) + : : : +- CometBroadcastHashJoin (83) + : : : :- CometProject (78) + : : : : +- CometBroadcastHashJoin (77) + : : : : :- CometProject (75) + : : : : : +- CometBroadcastHashJoin (74) + : : : : : :- CometProject (72) + : : : : : : +- CometBroadcastHashJoin (71) + : : : : : : :- CometFilter (69) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : : : : +- ReusedExchange (70) + : : : : : +- ReusedExchange (73) + : : : : +- ReusedExchange (76) + : : : +- CometBroadcastExchange (82) + : : : +- CometProject (81) + : : : +- CometFilter (80) + : : : +- CometScan parquet spark_catalog.default.customer_address (79) + : : +- ReusedExchange (85) + : +- ReusedExchange (88) + :- * HashAggregate (123) + : +- * CometColumnarToRow (122) + : +- CometColumnarExchange (121) + : +- * HashAggregate (120) + : +- * CometColumnarToRow (119) + : +- CometProject (118) + : +- CometBroadcastHashJoin (117) + : :- CometProject (115) + : : +- CometBroadcastHashJoin (114) + : : :- CometProject (112) + : : : +- CometBroadcastHashJoin (111) + : : : :- CometProject (106) + : : : : +- CometBroadcastHashJoin (105) + : : : : :- CometProject (103) + : : : : : +- CometBroadcastHashJoin (102) + : : : : : :- CometProject (100) + : : : : : : +- CometBroadcastHashJoin (99) + : : : : : : :- CometFilter (97) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (96) + : : : : : : +- ReusedExchange (98) + : : : : : +- ReusedExchange (101) + : : : : +- ReusedExchange (104) + : : : +- CometBroadcastExchange (110) + : : : +- CometProject (109) + : : : +- CometFilter (108) + : : : +- CometScan parquet spark_catalog.default.customer_address (107) + : : +- ReusedExchange (113) + : +- ReusedExchange (116) + +- * HashAggregate (150) + +- * CometColumnarToRow (149) + +- CometColumnarExchange (148) + +- * HashAggregate (147) + +- * CometColumnarToRow (146) + +- CometProject (145) + +- CometBroadcastHashJoin (144) + :- CometProject (140) + : +- CometBroadcastHashJoin (139) + : :- CometProject (137) + : : +- CometBroadcastHashJoin (136) + : : :- CometProject (134) + : : : +- CometBroadcastHashJoin (133) + : : : :- CometProject (131) + : : : : +- CometBroadcastHashJoin (130) + : : : : :- CometProject (128) + : : : : : +- CometBroadcastHashJoin (127) + : : : : : :- CometFilter (125) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (124) + : : : : : +- ReusedExchange (126) + : : : : +- ReusedExchange (129) + : : : +- ReusedExchange (132) + : : +- ReusedExchange (135) + : +- ReusedExchange (138) + +- CometBroadcastExchange (143) + +- CometFilter (142) + +- CometScan parquet spark_catalog.default.item (141) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -316,7 +321,7 @@ Arguments: [cs_item_sk#3], [i_item_sk#27], Inner, BuildRight Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#27, i_item_id#28] Arguments: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35], [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#29, cast(cs_list_price#5 as decimal(12,2)) AS agg2#30, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#31, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#32, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#33, cast(c_birth_year#19 as decimal(12,2)) AS agg6#34, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#35] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [11]: [i_item_id#28, ca_country#24, ca_state#23, ca_county#22, agg1#29, agg2#30, agg3#31, agg4#32, agg5#33, agg6#34, agg7#35] (37) HashAggregate [codegen id : 1] @@ -326,18 +331,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#28, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(39) HashAggregate [codegen id : 2] +(39) CometColumnarToRow [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] + +(40) 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) CometScan parquet spark_catalog.default.catalog_sales +(41) CometScan 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 +353,119 @@ 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 +(42) 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] +(43) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#88, cd_dep_count#89] -(43) CometBroadcastHashJoin +(44) 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 +(45) 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] +(46) 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 +(47) 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 +(48) 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] +(49) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#94] -(49) CometBroadcastHashJoin +(50) 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 +(51) 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) CometScan parquet spark_catalog.default.customer_address +(52) CometScan 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 +(53) 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 +(54) 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 +(55) 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 +(56) 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] +(57) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#98] -(57) CometBroadcastHashJoin +(58) 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 +(59) 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] +(60) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#99, i_item_id#100] -(60) CometBroadcastHashJoin +(61) 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 +(62) 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] +(63) CometColumnarToRow [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] +(64) 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 +(65) 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] + +(66) CometColumnarToRow [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] +(67) 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) CometScan parquet spark_catalog.default.catalog_sales +(68) CometScan 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 +473,123 @@ 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 +(69) 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] +(70) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#161, cd_dep_count#162] -(69) CometBroadcastHashJoin +(71) 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 +(72) 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] +(73) 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 +(74) 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 +(75) 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] +(76) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#167] -(75) CometBroadcastHashJoin +(77) 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 +(78) 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) CometScan parquet spark_catalog.default.customer_address +(79) CometScan 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 +(80) 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 +(81) 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 +(82) CometBroadcastExchange Input [2]: [ca_address_sk#168, ca_country#170] Arguments: [ca_address_sk#168, ca_country#170] -(81) CometBroadcastHashJoin +(83) 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 +(84) 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] +(85) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#171] -(84) CometBroadcastHashJoin +(86) 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 +(87) 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] +(88) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#172, i_item_id#173] -(87) CometBroadcastHashJoin +(89) 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 +(90) 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] +(91) CometColumnarToRow [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] +(92) 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 +(93) 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] +(94) CometColumnarToRow [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] + +(95) 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) CometScan parquet spark_catalog.default.catalog_sales +(96) CometScan 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 +597,123 @@ 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 +(97) 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] +(98) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#235, cd_dep_count#236] -(96) CometBroadcastHashJoin +(99) 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 +(100) 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] +(101) 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 +(102) 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 +(103) 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] +(104) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#241] -(102) CometBroadcastHashJoin +(105) 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 +(106) 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) CometScan parquet spark_catalog.default.customer_address +(107) CometScan 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 +(108) 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 +(109) CometProject Input [2]: [ca_address_sk#242, ca_state#243] Arguments: [ca_address_sk#242], [ca_address_sk#242] -(107) CometBroadcastExchange +(110) CometBroadcastExchange Input [1]: [ca_address_sk#242] Arguments: [ca_address_sk#242] -(108) CometBroadcastHashJoin +(111) 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 +(112) 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] +(113) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#244] -(111) CometBroadcastHashJoin +(114) 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 +(115) CometProject Input [10]: [cs_item_sk#227, cs_quantity#228, cs_list_price#229, cs_sales_price#230, cs_coupon_amt#231, cs_net_profit#232, 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] +(116) ReusedExchange [Reuses operator id: 33] Output [2]: [i_item_sk#245, i_item_id#246] -(114) CometBroadcastHashJoin +(117) 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 +(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, 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] +(119) CometColumnarToRow [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] +(120) 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 +(121) 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] +(122) CometColumnarToRow [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] + +(123) 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) CometScan parquet spark_catalog.default.catalog_sales +(124) CometScan 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 +721,162 @@ 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 +(125) 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] +(126) ReusedExchange [Reuses operator id: 6] Output [2]: [cd_demo_sk#309, cd_dep_count#310] -(123) CometBroadcastHashJoin +(127) 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 +(128) 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] +(129) 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 +(130) 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 +(131) 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] +(132) ReusedExchange [Reuses operator id: 17] Output [1]: [cd_demo_sk#315] -(129) CometBroadcastHashJoin +(133) 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 +(134) 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] +(135) ReusedExchange [Reuses operator id: 110] Output [1]: [ca_address_sk#316] -(132) CometBroadcastHashJoin +(136) 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 +(137) 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] +(138) ReusedExchange [Reuses operator id: 28] Output [1]: [d_date_sk#317] -(135) CometBroadcastHashJoin +(139) 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 +(140) 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) CometScan parquet spark_catalog.default.item +(141) CometScan 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 +(142) CometFilter Input [1]: [i_item_sk#318] Condition : isnotnull(i_item_sk#318) -(139) CometBroadcastExchange +(143) CometBroadcastExchange Input [1]: [i_item_sk#318] Arguments: [i_item_sk#318] -(140) CometBroadcastHashJoin +(144) 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 +(145) 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] +(146) CometColumnarToRow [codegen id : 9] Input [7]: [agg1#319, agg2#320, agg3#321, agg4#322, agg5#323, agg6#324, agg7#325] -(143) HashAggregate [codegen id : 9] +(147) 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 +(148) 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] + +(149) CometColumnarToRow [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] +(150) 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 +(151) Union -(147) TakeOrderedAndProject +(152) 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 (157) ++- * CometColumnarToRow (156) + +- CometProject (155) + +- CometFilter (154) + +- CometScan parquet spark_catalog.default.date_dim (153) -(148) CometScan parquet spark_catalog.default.date_dim +(153) CometScan 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 +(154) 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 +(155) CometProject Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(151) ColumnarToRow [codegen id : 1] +(156) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] -(152) BroadcastExchange +(157) 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 = 41 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 = 68 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 = 96 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 = 124 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 509d4be06c..3925a5af93 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,183 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + 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 6a93509c8e..2bdbfa65e7 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,28 @@ == 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 (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.catalog_sales @@ -87,7 +89,7 @@ Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -97,66 +99,72 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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 +(24) 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) -(23) CometScan parquet spark_catalog.default.date_dim +(25) CometScan 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 +(26) CometFilter Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#12 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) -(25) CometProject +(27) CometProject Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(27) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#11] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=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 cd6bcfd611..322cc4e0d0 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,37 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 34d510fc71..1afbec79bb 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,29 @@ == 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 (25) ++- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastNestedLoopJoin Inner BuildRight (18) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (16) + +- CometScan parquet spark_catalog.default.warehouse (15) (1) CometScan parquet spark_catalog.default.inventory @@ -89,7 +90,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +99,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -124,50 +125,53 @@ 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) 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, [plan_id=2] +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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarToRow [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] + +(24) 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 +(25) 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 (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(25) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(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)) -(27) CometProject +(28) 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] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(30) 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 0e864ab701..7fed747fe6 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,40 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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 + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 4cabf98776..8bc821272e 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,59 @@ == 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) - :- * HashAggregate (34) - : +- Exchange (33) - : +- * HashAggregate (32) - : +- * HashAggregate (31) - : +- ReusedExchange (30) +TakeOrderedAndProject (55) ++- Union (54) + :- * HashAggregate (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (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 (32) + : +- * CometColumnarToRow (31) + : +- CometColumnarExchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- * CometColumnarToRow (27) + : +- ReusedExchange (26) :- * HashAggregate (39) - : +- Exchange (38) - : +- * HashAggregate (37) + : +- * CometColumnarToRow (38) + : +- CometColumnarExchange (37) : +- * HashAggregate (36) - : +- ReusedExchange (35) - +- * HashAggregate (44) - +- Exchange (43) - +- * HashAggregate (42) - +- * HashAggregate (41) - +- ReusedExchange (40) + : +- * HashAggregate (35) + : +- * CometColumnarToRow (34) + : +- ReusedExchange (33) + :- * HashAggregate (46) + : +- * CometColumnarToRow (45) + : +- CometColumnarExchange (44) + : +- * HashAggregate (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- ReusedExchange (40) + +- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- ReusedExchange (47) (1) CometScan parquet spark_catalog.default.inventory @@ -135,7 +144,7 @@ Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#13], Inner, BuildRight Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12, w_warehouse_sk#13] Arguments: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12], [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [inv_quantity_on_hand#3, i_brand#9, i_class#10, i_category#11, i_product_name#12] (20) HashAggregate [codegen id : 1] @@ -145,178 +154,205 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_product_name#12, i_brand#9, i_class#10, i_category#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [codegen id : 2] +Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#16, count#17] + +(23) 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] +(24) 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] +(25) 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] +(26) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] -(26) HashAggregate [codegen id : 4] +(27) CometColumnarToRow [codegen id : 4] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] + +(28) 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] +(29) 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 +(30) 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] + +(31) CometColumnarToRow [codegen id : 5] 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) 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] +(33) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] -(31) HashAggregate [codegen id : 7] +(34) CometColumnarToRow [codegen id : 7] +Input [6]: [i_product_name#45, i_brand#46, i_class#47, i_category#48, sum#49, count#50] + +(35) 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] +(36) 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 +(37) 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, [plan_id=3] +Arguments: hashpartitioning(i_product_name#45, i_brand#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) HashAggregate [codegen id : 8] +(38) CometColumnarToRow [codegen id : 8] +Input [4]: [i_product_name#45, i_brand#46, sum#55, count#56] + +(39) 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] +(40) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] -(36) HashAggregate [codegen id : 10] +(41) CometColumnarToRow [codegen id : 10] +Input [6]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, sum#65, count#66] + +(42) 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] +(43) 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 +(44) CometColumnarExchange +Input [3]: [i_product_name#61, sum#71, count#72] +Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(45) CometColumnarToRow [codegen id : 11] Input [3]: [i_product_name#61, sum#71, count#72] -Arguments: hashpartitioning(i_product_name#61, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(39) HashAggregate [codegen id : 11] +(46) 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] +(47) ReusedExchange [Reuses operator id: 21] Output [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] -(41) HashAggregate [codegen id : 13] +(48) CometColumnarToRow [codegen id : 13] +Input [6]: [i_product_name#78, i_brand#79, i_class#80, i_category#81, sum#82, count#83] + +(49) 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] +(50) 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 +(51) CometColumnarExchange +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(52) CometColumnarToRow [codegen id : 14] Input [2]: [sum#88, count#89] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] -(44) HashAggregate [codegen id : 14] +(53) 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 +(54) Union -(46) TakeOrderedAndProject +(55) 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 (60) ++- * CometColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) -(47) CometScan parquet spark_catalog.default.date_dim +(56) CometScan 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 +(57) 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 +(58) 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] +(59) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(51) BroadcastExchange +(60) 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 fc4778a226..8c09280baf 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,80 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + 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 522e028b5a..0e031ade47 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,49 +1,51 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometSort (44) - +- CometColumnarExchange (43) - +- * Filter (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* CometColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- * Filter (44) + +- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -177,7 +179,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] +(29) CometColumnarToRow [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] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +193,7 @@ ReadSchema: struct cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(43) CometColumnarExchange +(45) 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, CometColumnarShuffle, [plan_id=6] -(44) CometSort +(46) 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] -(45) ColumnarToRow [codegen id : 5] +(47) CometColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (72) -+- Exchange (71) - +- * HashAggregate (70) - +- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * ColumnarToRow (63) - : +- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (59) - : : +- CometBroadcastHashJoin (58) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (52) - : : +- CometBroadcastExchange (57) - : : +- CometFilter (56) - : : +- CometScan parquet spark_catalog.default.item (55) - : +- ReusedExchange (60) - +- ReusedExchange (64) - - -(46) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (76) ++- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * CometColumnarToRow (71) + +- CometColumnarExchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometProject (53) + : : : : +- CometSortMergeJoin (52) + : : : : :- CometSort (49) + : : : : : +- ReusedExchange (48) + : : : : +- CometSort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.item (57) + : +- ReusedExchange (62) + +- ReusedExchange (66) + + +(48) 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] -(47) CometSort +(49) 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_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] -(48) ReusedExchange [Reuses operator id: 9] +(50) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#47, sr_ticket_number#48] -(49) CometSort +(51) CometSort Input [2]: [sr_item_sk#47, sr_ticket_number#48] Arguments: [sr_item_sk#47, sr_ticket_number#48], [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST] -(50) CometSortMergeJoin +(52) 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 -(51) CometProject +(53) 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] -(52) ReusedExchange [Reuses operator id: 16] +(54) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] -(53) CometBroadcastHashJoin +(55) 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 -(54) CometProject +(56) 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] -(55) CometScan parquet spark_catalog.default.item +(57) CometScan 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 -(56) CometFilter +(58) 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) -(57) CometBroadcastExchange +(59) 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] -(58) CometBroadcastHashJoin +(60) 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 -(59) CometProject +(61) 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] -(60) ReusedExchange [Reuses operator id: 26] +(62) 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] -(61) CometBroadcastHashJoin +(63) 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 -(62) CometProject +(64) 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] -(63) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [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] -(64) ReusedExchange [Reuses operator id: 33] +(66) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] -(65) BroadcastHashJoin [codegen id : 2] +(67) 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 -(66) Project [codegen id : 2] +(68) 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] -(67) HashAggregate [codegen id : 2] +(69) 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] -(68) Exchange +(70) 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, CometColumnarShuffle, [plan_id=7] + +(71) CometColumnarToRow [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] -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=7] -(69) HashAggregate [codegen id : 3] +(72) 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] -(70) HashAggregate [codegen id : 3] +(73) 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] -(71) Exchange +(74) CometColumnarExchange +Input [2]: [sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(75) CometColumnarToRow [codegen id : 4] Input [2]: [sum#73, count#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(72) HashAggregate [codegen id : 4] +(76) 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 f4bcae0f13..27e6a9cd77 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 @@ -8,18 +8,55 @@ WholeStageCodegen (5) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + 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] + CometColumnarToRow 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 + 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 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 + CometColumnarToRow 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] @@ -30,62 +67,29 @@ WholeStageCodegen (5) 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 + CometExchange [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] - 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 + CometExchange [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] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + 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 - 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 (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] - 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 (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] - CometExchange [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] - CometExchange [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 #9 - WholeStageCodegen (1) - 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] + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + 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] 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 42a49492f0..46192f2659 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,78 @@ == 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 (74) ++- Union (73) + :- * HashAggregate (29) + : +- * CometColumnarToRow (28) + : +- CometColumnarExchange (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (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 (51) + : +- * CometColumnarToRow (50) + : +- CometColumnarExchange (49) + : +- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (43) + : : +- CometBroadcastHashJoin (42) + : : :- CometProject (37) + : : : +- CometBroadcastHashJoin (36) + : : : :- CometProject (34) + : : : : +- CometBroadcastHashJoin (33) + : : : : :- CometFilter (31) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (32) + : : : +- ReusedExchange (35) + : : +- CometBroadcastExchange (41) + : : +- CometProject (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.store (38) + : +- ReusedExchange (44) + +- * HashAggregate (72) + +- * CometColumnarToRow (71) + +- CometColumnarExchange (70) + +- * HashAggregate (69) + +- * CometColumnarToRow (68) + +- CometProject (67) + +- CometBroadcastHashJoin (66) + :- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometProject (59) + : : +- CometBroadcastHashJoin (58) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.store_sales (52) + : : : +- ReusedExchange (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (60) + +- CometBroadcastExchange (65) + +- CometFilter (64) + +- CometScan parquet spark_catalog.default.item (63) (1) CometScan parquet spark_catalog.default.store_sales @@ -188,7 +191,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#18], Inner, BuildRight Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#19] Arguments: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23], [i_item_id#19, s_state#17, ss_quantity#4 AS agg1#20, ss_list_price#5 AS agg2#21, ss_coupon_amt#7 AS agg3#22, ss_sales_price#6 AS agg4#23] -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_id#19, s_state#17, agg1#20, agg2#21, agg3#22, agg4#23] (26) HashAggregate [codegen id : 1] @@ -198,18 +201,21 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_item_id#19, s_state#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [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] + +(29) 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) CometScan parquet spark_catalog.default.store_sales +(30) CometScan 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 +223,99 @@ 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 +(31) 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] +(32) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#58] -(32) CometBroadcastHashJoin +(33) 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 +(34) 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] +(35) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#59] -(35) CometBroadcastHashJoin +(36) 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 +(37) 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) CometScan parquet spark_catalog.default.store +(38) CometScan 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 +(39) 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 +(40) CometProject Input [2]: [s_store_sk#60, s_state#61] Arguments: [s_store_sk#60], [s_store_sk#60] -(40) CometBroadcastExchange +(41) CometBroadcastExchange Input [1]: [s_store_sk#60] Arguments: [s_store_sk#60] -(41) CometBroadcastHashJoin +(42) 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 +(43) 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] +(44) ReusedExchange [Reuses operator id: 22] Output [2]: [i_item_sk#62, i_item_id#63] -(44) CometBroadcastHashJoin +(45) 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 +(46) 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] +(47) CometColumnarToRow [codegen id : 3] Input [5]: [i_item_id#63, agg1#64, agg2#65, agg3#66, agg4#67] -(47) HashAggregate [codegen id : 3] +(48) 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 +(49) 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, CometColumnarShuffle, [plan_id=2] + +(50) CometColumnarToRow [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] -Arguments: hashpartitioning(i_item_id#63, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(49) HashAggregate [codegen id : 4] +(51) 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) CometScan parquet spark_catalog.default.store_sales +(52) CometScan 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 +323,134 @@ 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 +(53) 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] +(54) ReusedExchange [Reuses operator id: 6] Output [1]: [cd_demo_sk#103] -(53) CometBroadcastHashJoin +(55) 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 +(56) 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] +(57) ReusedExchange [Reuses operator id: 12] Output [1]: [d_date_sk#104] -(56) CometBroadcastHashJoin +(58) 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 +(59) 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] +(60) ReusedExchange [Reuses operator id: 41] Output [1]: [s_store_sk#105] -(59) CometBroadcastHashJoin +(61) 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 +(62) 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) CometScan parquet spark_catalog.default.item +(63) CometScan 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 +(64) CometFilter Input [1]: [i_item_sk#106] Condition : isnotnull(i_item_sk#106) -(63) CometBroadcastExchange +(65) CometBroadcastExchange Input [1]: [i_item_sk#106] Arguments: [i_item_sk#106] -(64) CometBroadcastHashJoin +(66) 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 +(67) 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] +(68) CometColumnarToRow [codegen id : 5] Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] -(67) HashAggregate [codegen id : 5] +(69) 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 +(70) 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] + +(71) CometColumnarToRow [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] +(72) 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 +(73) Union -(71) TakeOrderedAndProject +(74) 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 (79) ++- * CometColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) -(72) CometScan parquet spark_catalog.default.date_dim +(75) CometScan 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 +(76) 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 +(77) CometProject Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(75) ColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(76) BroadcastExchange +(79) 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 = 30 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 = 52 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 50e246e15c..913c4ed5ad 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,95 @@ 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] - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] - ReusedExchange [i_item_sk,i_item_id] #6 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + WholeStageCodegen (5) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + CometColumnarToRow + 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 2d2b384c87..25e9930cce 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 7da6076f5b..e1fc6bf43f 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 5933ea5fad..cc7702fe7b 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 0b55e23ab6..82c1f313df 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 df30176f06..aa18d076f1 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,43 @@ == 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 (39) ++- * HashAggregate (38) + +- * CometColumnarToRow (37) + +- CometColumnarExchange (36) + +- * HashAggregate (35) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.customer @@ -197,7 +198,7 @@ Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Arguments: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -(34) ColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [6]: [ca_state#21, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] (35) HashAggregate [codegen id : 1] @@ -207,50 +208,53 @@ 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) 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, [plan_id=1] +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] -(37) HashAggregate [codegen id : 2] +(37) CometColumnarToRow [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] + +(38) 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 +(39) 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 (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) -(39) CometScan parquet spark_catalog.default.date_dim +(40) CometScan 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 +(41) 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 +(42) 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] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(43) BroadcastExchange +(44) 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 79ead04645..49c76cd13b 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,53 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 17aa8a6357..01a04c9975 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,52 @@ == 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 (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometExchange (43) + +- CometHashAggregate (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- Union (39) + :- * HashAggregate (24) + : +- * CometColumnarToRow (23) + : +- CometColumnarExchange (22) + : +- * HashAggregate (21) + : +- * CometColumnarToRow (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 (31) + : +- * CometColumnarToRow (30) + : +- CometColumnarExchange (29) + : +- * HashAggregate (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- ReusedExchange (25) + +- * HashAggregate (38) + +- * CometColumnarToRow (37) + +- CometColumnarExchange (36) + +- * HashAggregate (35) + +- * HashAggregate (34) + +- * CometColumnarToRow (33) + +- ReusedExchange (32) (1) CometScan parquet spark_catalog.default.store_sales @@ -135,7 +141,7 @@ Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11, s_store_sk#12] Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11], [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] -(20) ColumnarToRow [codegen id : 1] +(20) CometColumnarToRow [codegen id : 1] Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#10, i_category#11] (21) HashAggregate [codegen id : 1] @@ -145,142 +151,158 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(23) HashAggregate [codegen id : 2] +(23) CometColumnarToRow [codegen id : 2] +Input [4]: [i_category#11, i_class#10, sum#16, sum#17] + +(24) 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] +(25) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#24, i_class#25, sum#26, sum#27] -(25) HashAggregate [codegen id : 4] +(26) CometColumnarToRow [codegen id : 4] +Input [4]: [i_category#24, i_class#25, sum#26, sum#27] + +(27) 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] +(28) 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 +(29) CometColumnarExchange Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(28) HashAggregate [codegen id : 5] +(30) CometColumnarToRow [codegen id : 5] +Input [5]: [i_category#24, sum#38, isEmpty#39, sum#40, isEmpty#41] + +(31) 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] +(32) ReusedExchange [Reuses operator id: 22] Output [4]: [i_category#49, i_class#50, sum#51, sum#52] -(30) HashAggregate [codegen id : 7] +(33) CometColumnarToRow [codegen id : 7] +Input [4]: [i_category#49, i_class#50, sum#51, sum#52] + +(34) 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] +(35) 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 +(36) CometColumnarExchange +Input [4]: [sum#61, isEmpty#62, sum#63, isEmpty#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(37) CometColumnarToRow [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] +(38) 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 +(39) Union -(35) HashAggregate [codegen id : 9] +(40) 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 +(41) 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, [plan_id=4] +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] -(37) HashAggregate [codegen id : 10] +(42) 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 +(43) CometExchange +Input [5]: [gross_margin#20, i_category#11, i_class#10, lochierarchy#23, _w0#73] +Arguments: hashpartitioning(lochierarchy#23, _w0#73, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(44) 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] +(45) CometColumnarToRow [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 +(46) 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] +(47) 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 +(48) 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 (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) -(43) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 +(50) 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 +(51) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(47) BroadcastExchange +(53) 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 59c22c3359..7ed5146a60 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,27 +1,27 @@ 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) + CometColumnarToRow 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] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [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 + 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] + CometColumnarToRow InputAdapter - Exchange [i_category,i_class] #3 + CometColumnarExchange [i_category,i_class] #3 WholeStageCodegen (1) HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow 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] @@ -34,7 +34,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -50,21 +50,25 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i 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] + 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] + CometColumnarToRow InputAdapter - Exchange [i_category] #8 + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #9 + CometColumnarExchange #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 + CometColumnarToRow + 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 51250f6234..46df5a8c4c 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,54 @@ == 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 (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (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 (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan parquet spark_catalog.default.item @@ -130,7 +135,7 @@ Arguments: [ss_store_sk#5], [s_store_sk#12], Inner, BuildRight Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] Arguments: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14], [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] (19) HashAggregate [codegen id : 1] @@ -140,142 +145,157 @@ 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) 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, [plan_id=1] +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] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [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] + +(22) 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 +(23) 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, CometColumnarShuffle, [plan_id=2] + +(24) 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: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] +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] -(23) Sort [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(24) Window +(26) 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] +(27) 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 +(28) 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] +(29) 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] +(30) 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] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(30) HashAggregate [codegen id : 6] +(32) CometColumnarToRow [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] + +(33) 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 +(34) 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, CometColumnarShuffle, [plan_id=3] + +(35) 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: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=3] +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] -(32) Sort [codegen id : 7] +(36) CometColumnarToRow [codegen id : 7] Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 -(33) Window +(37) 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] +(38) 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 +(39) 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] +(40) 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] +(41) 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] +(42) ReusedExchange [Reuses operator id: 34] 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] +(43) 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, 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] + +(44) CometColumnarToRow [codegen id : 11] Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 -(40) Window +(45) 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] +(46) 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 +(47) 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] +(48) 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] +(49) 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 +(50) 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 (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(46) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(54) 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 61f4976b4a..1bbf065599 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,43 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 +57,15 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + 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 +73,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] + CometColumnarToRow 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 138876bfcf..2fc63b2fea 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,88 @@ == 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) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- Union (79) + :- * Project (28) + : +- * Filter (27) + : +- Window (26) + : +- * Sort (25) + : +- Window (24) + : +- * CometColumnarToRow (23) + : +- CometSort (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (19) + : +- CometColumnarExchange (18) + : +- * HashAggregate (17) + : +- * CometColumnarToRow (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 (53) + : +- * Filter (52) + : +- Window (51) + : +- * Sort (50) + : +- Window (49) + : +- * CometColumnarToRow (48) + : +- CometSort (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * CometColumnarToRow (44) + : +- CometColumnarExchange (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometBroadcastExchange (32) + : : : +- CometProject (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.catalog_returns (33) + : +- ReusedExchange (38) + +- * Project (78) + +- * Filter (77) + +- Window (76) + +- * Sort (75) + +- Window (74) + +- * CometColumnarToRow (73) + +- CometSort (72) + +- CometColumnarExchange (71) + +- * HashAggregate (70) + +- * CometColumnarToRow (69) + +- CometColumnarExchange (68) + +- * HashAggregate (67) + +- * CometColumnarToRow (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometBroadcastExchange (57) + : : +- CometProject (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.store_sales (54) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometScan parquet spark_catalog.default.store_returns (58) + +- ReusedExchange (63) (1) CometScan parquet spark_catalog.default.web_sales @@ -150,7 +157,7 @@ Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] (17) HashAggregate [codegen id : 1] @@ -160,46 +167,52 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(19) HashAggregate [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] +Input [7]: [ws_item_sk#1, sum#22, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] + +(20) 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 +(21) CometColumnarExchange +Input [3]: [item#32, return_ratio#33, currency_ratio#34] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(22) CometSort Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: [item#32, return_ratio#33, currency_ratio#34], [return_ratio#33 ASC NULLS FIRST] -(21) Sort [codegen id : 3] +(23) CometColumnarToRow [codegen id : 3] Input [3]: [item#32, return_ratio#33, currency_ratio#34] -Arguments: [return_ratio#33 ASC NULLS FIRST], false, 0 -(22) Window +(24) 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] +(25) 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 +(26) 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] +(27) 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] +(28) 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) CometScan parquet spark_catalog.default.catalog_sales +(29) CometScan 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 +220,110 @@ 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 +(30) 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 +(31) 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 +(32) 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) CometScan parquet spark_catalog.default.catalog_returns +(33) CometScan 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 +(34) 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 +(35) 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 +(36) 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 +(37) 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] +(38) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#50] -(37) CometBroadcastHashJoin +(39) 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 +(40) 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] +(41) CometColumnarToRow [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] +(42) 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 +(43) 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] + +(44) CometColumnarToRow [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] +(45) 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 +(46) CometColumnarExchange +Input [3]: [item#67, return_ratio#68, currency_ratio#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(47) 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] +(48) CometColumnarToRow [codegen id : 8] Input [3]: [item#67, return_ratio#68, currency_ratio#69] -Arguments: [return_ratio#68 ASC NULLS FIRST], false, 0 -(45) Window +(49) 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] +(50) 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 +(51) 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] +(52) 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] +(53) 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) CometScan parquet spark_catalog.default.store_sales +(54) CometScan 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 +331,168 @@ 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 +(55) 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 +(56) 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 +(57) 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) CometScan parquet spark_catalog.default.store_returns +(58) CometScan 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 +(59) 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 +(60) 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 +(61) 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 +(62) 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] +(63) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#85] -(60) CometBroadcastHashJoin +(64) 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 +(65) 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] +(66) CometColumnarToRow [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] +(67) 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 +(68) 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, CometColumnarShuffle, [plan_id=5] + +(69) CometColumnarToRow [codegen id : 12] Input [7]: [ss_item_sk#73, sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(ss_item_sk#73, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(65) HashAggregate [codegen id : 12] +(70) 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 +(71) 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] +(72) 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 +(73) CometColumnarToRow [codegen id : 13] +Input [3]: [item#102, return_ratio#103, currency_ratio#104] + +(74) 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] +(75) 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 +(76) 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] +(77) 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] +(78) 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 +(79) Union -(74) HashAggregate [codegen id : 16] +(80) 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 +(81) 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] +(82) 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 +(83) 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] + +(84) CometColumnarToRow [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 (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan parquet spark_catalog.default.date_dim (85) -(78) CometScan parquet spark_catalog.default.date_dim +(85) CometScan 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 +(86) 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 +(87) 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] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(89) 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 = 29 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 = 54 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 bd3b21cdbd..69f6f217e6 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,128 @@ -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] - 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] - 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] - 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 +WholeStageCodegen (17) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometColumnarExchange #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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 af53d08ccc..5acbb9781f 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,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) - : : : +- * 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 (80) ++- * Filter (79) + +- * HashAggregate (78) + +- * HashAggregate (77) + +- * Project (76) + +- * BroadcastHashJoin Inner BuildRight (75) + :- Window (68) + : +- * CometColumnarToRow (67) + : +- CometSort (66) + : +- CometExchange (65) + : +- CometProject (64) + : +- CometFilter (63) + : +- CometSortMergeJoin (62) + : :- CometSort (32) + : : +- CometColumnarExchange (31) + : : +- * HashAggregate (30) + : : +- * CometColumnarToRow (29) + : : +- CometColumnarExchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (18) + : : : +- Window (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometSort (15) + : : : +- CometColumnarExchange (14) + : : : +- * HashAggregate (13) + : : : +- * CometColumnarToRow (12) + : : : +- CometColumnarExchange (11) + : : : +- * HashAggregate (10) + : : : +- * CometColumnarToRow (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 (24) + : : +- * Project (23) + : : +- Window (22) + : : +- * CometColumnarToRow (21) + : : +- CometSort (20) + : : +- ReusedExchange (19) + : +- CometSort (61) + : +- CometColumnarExchange (60) + : +- * HashAggregate (59) + : +- * CometColumnarToRow (58) + : +- CometColumnarExchange (57) + : +- * HashAggregate (56) + : +- * Project (55) + : +- * BroadcastHashJoin Inner BuildRight (54) + : :- * Project (47) + : : +- Window (46) + : : +- * CometColumnarToRow (45) + : : +- CometSort (44) + : : +- CometColumnarExchange (43) + : : +- * HashAggregate (42) + : : +- * CometColumnarToRow (41) + : : +- CometColumnarExchange (40) + : : +- * HashAggregate (39) + : : +- * CometColumnarToRow (38) + : : +- CometProject (37) + : : +- CometBroadcastHashJoin (36) + : : :- CometFilter (34) + : : : +- CometScan parquet spark_catalog.default.store_sales (33) + : : +- ReusedExchange (35) + : +- BroadcastExchange (53) + : +- * Project (52) + : +- Window (51) + : +- * CometColumnarToRow (50) + : +- CometSort (49) + : +- ReusedExchange (48) + +- BroadcastExchange (74) + +- * Project (73) + +- Window (72) + +- * CometColumnarToRow (71) + +- CometSort (70) + +- ReusedExchange (69) (1) CometScan parquet spark_catalog.default.web_sales @@ -111,7 +121,7 @@ Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] -(9) ColumnarToRow [codegen id : 1] +(9) CometColumnarToRow [codegen id : 1] Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] (10) HashAggregate [codegen id : 1] @@ -121,89 +131,101 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(12) HashAggregate [codegen id : 2] +(12) CometColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, d_date#6, sum#9] + +(13) 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 +(14) 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] +(15) 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 +(16) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#11, d_date#6, sumws#12, ws_item_sk#1] + +(17) 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] +(18) 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] +(19) ReusedExchange [Reuses operator id: 14] Output [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] -(18) Sort [codegen id : 6] +(20) CometSort +Input [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] +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] + +(21) CometColumnarToRow [codegen id : 6] Input [4]: [item_sk#14, d_date#15, sumws#16, ws_item_sk#17] -Arguments: [ws_item_sk#17 ASC NULLS FIRST, d_date#15 ASC NULLS FIRST], false, 0 -(19) Window +(22) 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] +(23) 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 +(24) 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] +(25) 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] +(26) 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] +(27) 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 +(28) 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] + +(29) CometColumnarToRow [codegen id : 9] 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] +(30) 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 +(31) 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] +(32) 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) CometScan parquet spark_catalog.default.store_sales +(33) CometScan 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 +233,240 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter +(34) 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] +(35) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#29, d_date#30] -(32) CometBroadcastHashJoin +(36) 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 +(37) 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] +(38) CometColumnarToRow [codegen id : 10] Input [3]: [ss_item_sk#25, ss_sales_price#26, d_date#30] -(35) HashAggregate [codegen id : 11] +(39) 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 +(40) 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] + +(41) CometColumnarToRow [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] +(42) 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 +(43) 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] +(44) 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 +(45) CometColumnarToRow [codegen id : 12] +Input [4]: [item_sk#34, d_date#30, sumss#35, ss_item_sk#25] + +(46) 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] +(47) 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] +(48) ReusedExchange [Reuses operator id: 43] Output [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] -(43) Sort [codegen id : 16] +(49) 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 +(50) CometColumnarToRow [codegen id : 15] +Input [4]: [item_sk#37, d_date#38, sumss#39, ss_item_sk#40] + +(51) 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] +(52) 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 +(53) 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] +(54) 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] +(55) 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] +(56) 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 +(57) 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] + +(58) CometColumnarToRow [codegen id : 18] 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] +(59) 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 +(60) 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] +(61) 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 +(62) 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] +(63) 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] +(64) 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] + +(65) CometExchange +Input [4]: [item_sk#48, d_date#49, web_sales#50, store_sales#51] +Arguments: hashpartitioning(item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(57) Exchange +(66) CometSort 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: [item_sk#48, d_date#49, web_sales#50, store_sales#51], [item_sk#48 ASC NULLS FIRST, d_date#49 ASC NULLS FIRST] -(58) Sort [codegen id : 22] +(67) CometColumnarToRow [codegen id : 19] 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 -(59) Window +(68) 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] +(69) ReusedExchange [Reuses operator id: 65] Output [4]: [item_sk#53, d_date#54, web_sales#55, store_sales#56] -(61) Sort [codegen id : 44] +(70) 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] + +(71) CometColumnarToRow [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 +(72) 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] +(73) 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 +(74) 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] +(75) 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] +(76) 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] +(77) 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] +(78) 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] +(79) 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 +(80) 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 (85) ++- * CometColumnarToRow (84) + +- CometProject (83) + +- CometFilter (82) + +- CometScan parquet spark_catalog.default.date_dim (81) -(71) CometScan parquet spark_catalog.default.date_dim +(81) CometScan 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) CometColumnarToRow [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#27 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 33 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 47ad5e34ff..7cb9582115 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,41 +7,41 @@ 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) + CometColumnarToRow 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) - 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] - InputAdapter - Exchange [ws_item_sk] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (8) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow InputAdapter - Exchange [ws_item_sk,d_date] #5 + CometColumnarExchange [ws_item_sk,d_date] #5 WholeStageCodegen (1) HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow 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] @@ -50,7 +50,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date] CometFilter [d_date_sk,d_date,d_month_seq] @@ -59,43 +59,44 @@ 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 - BroadcastExchange #8 - WholeStageCodegen (7) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (6) - Sort [ws_item_sk,d_date] - 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) - 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] - 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 + BroadcastExchange #8 + WholeStageCodegen (7) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + 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 + WholeStageCodegen (18) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + 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 (12) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + WholeStageCodegen (11) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow InputAdapter - Exchange [ss_item_sk,d_date] #12 - WholeStageCodegen (11) + CometColumnarExchange [ss_item_sk,d_date] #12 + WholeStageCodegen (10) HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - ColumnarToRow + CometColumnarToRow 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] @@ -103,23 +104,25 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (16) - Sort [ss_item_sk,d_date] - InputAdapter - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + 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) + CometColumnarToRow 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 ad7a09b49d..4dad6647e5 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,54 @@ == 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 (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (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 (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan parquet spark_catalog.default.item @@ -130,7 +135,7 @@ Arguments: [cs_call_center_sk#4], [cc_call_center_sk#12], Inner, BuildRight Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] Arguments: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13], [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] -(18) ColumnarToRow [codegen id : 1] +(18) CometColumnarToRow [codegen id : 1] Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] (19) HashAggregate [codegen id : 1] @@ -140,142 +145,157 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [codegen id : 2] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(22) 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 +(23) 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, CometColumnarShuffle, [plan_id=2] + +(24) CometSort 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: [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] -(23) Sort [codegen id : 3] +(25) CometColumnarToRow [codegen id : 3] Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] -Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 -(24) Window +(26) 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] +(27) 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 +(28) 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] +(29) 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] +(30) 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] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(30) HashAggregate [codegen id : 6] +(32) CometColumnarToRow [codegen id : 6] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(33) 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 +(34) 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, CometColumnarShuffle, [plan_id=3] + +(35) CometSort 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: [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] -(32) Sort [codegen id : 7] +(36) CometColumnarToRow [codegen id : 7] Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 -(33) Window +(37) 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] +(38) 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 +(39) 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] +(40) 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] +(41) 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] +(42) ReusedExchange [Reuses operator id: 34] 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] +(43) 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, 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] + +(44) CometColumnarToRow [codegen id : 11] Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 -(40) Window +(45) 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] +(46) 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 +(47) 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] +(48) 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] +(49) 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 +(50) 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 (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) -(46) CometScan parquet spark_catalog.default.date_dim +(51) CometScan 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 +(52) 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] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -(49) BroadcastExchange +(54) 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 daeafbf929..4883870e6d 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,43 @@ 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] + CometColumnarToRow 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 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 +57,15 @@ 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] + CometColumnarToRow 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 + WholeStageCodegen (6) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 WholeStageCodegen (12) @@ -69,6 +73,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] + CometColumnarToRow 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 4c45b9d4dd..d38e644d9a 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,96 @@ == 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) +* CometColumnarToRow (92) ++- CometTakeOrderedAndProject (91) + +- CometHashAggregate (90) + +- CometColumnarExchange (89) + +- * HashAggregate (88) + +- Union (87) + :- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- * HashAggregate (69) + : +- Union (68) + : :- * HashAggregate (23) + : : +- * CometColumnarToRow (22) + : : +- CometColumnarExchange (21) + : : +- * HashAggregate (20) + : : +- * CometColumnarToRow (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 (43) + : : +- * CometColumnarToRow (42) + : : +- CometColumnarExchange (41) + : : +- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometUnion (30) + : : : : :- CometProject (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : : : +- CometProject (29) + : : : : +- CometFilter (28) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (27) + : : : +- ReusedExchange (31) + : : +- CometBroadcastExchange (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.catalog_page (34) + : +- * HashAggregate (67) + : +- * CometColumnarToRow (66) + : +- CometColumnarExchange (65) + : +- * HashAggregate (64) + : +- * CometColumnarToRow (63) + : +- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometProject (57) + : : +- CometBroadcastHashJoin (56) + : : :- CometUnion (54) + : : : :- CometProject (46) + : : : : +- CometFilter (45) + : : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : : +- CometProject (53) + : : : +- CometBroadcastHashJoin (52) + : : : :- CometBroadcastExchange (48) + : : : : +- CometScan parquet spark_catalog.default.web_returns (47) + : : : +- CometProject (51) + : : : +- CometFilter (50) + : : : +- CometScan parquet spark_catalog.default.web_sales (49) + : : +- ReusedExchange (55) + : +- CometBroadcastExchange (60) + : +- CometFilter (59) + : +- CometScan parquet spark_catalog.default.web_site (58) + :- * HashAggregate (79) + : +- * CometColumnarToRow (78) + : +- CometColumnarExchange (77) + : +- * HashAggregate (76) + : +- * HashAggregate (75) + : +- * CometColumnarToRow (74) + : +- ReusedExchange (73) + +- * HashAggregate (86) + +- * CometColumnarToRow (85) + +- CometColumnarExchange (84) + +- * HashAggregate (83) + +- * HashAggregate (82) + +- * CometColumnarToRow (81) + +- ReusedExchange (80) (1) CometScan parquet spark_catalog.default.store_sales @@ -172,7 +181,7 @@ Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#25] Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] (20) HashAggregate [codegen id : 1] @@ -182,18 +191,21 @@ 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) CometColumnarExchange 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] +Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [codegen id : 2] +Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] + +(23) 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) CometScan parquet spark_catalog.default.catalog_sales +(24) CometScan 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 +213,15 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(24) CometFilter +(25) 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 +(26) 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) CometScan parquet spark_catalog.default.catalog_returns +(27) CometScan 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 +229,79 @@ PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(c PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(27) CometFilter +(28) 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 +(29) 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 +(30) 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] +(31) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#64] -(31) CometBroadcastHashJoin +(32) 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 +(33) 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) CometScan parquet spark_catalog.default.catalog_page +(34) CometScan 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 +(35) CometFilter Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(35) CometBroadcastExchange +(36) 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 +(37) 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 +(38) 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] +(39) CometColumnarToRow [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] +(40) 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 +(41) 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] + +(42) CometColumnarToRow [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] +(43) 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) CometScan parquet spark_catalog.default.web_sales +(44) CometScan 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 +309,269 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(43) CometFilter +(45) 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 +(46) 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) CometScan parquet spark_catalog.default.web_returns +(47) CometScan 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 +(48) 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) CometScan parquet spark_catalog.default.web_sales +(49) CometScan 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 +(50) 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 +(51) 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 +(52) 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 +(53) 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 +(54) 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] +(55) ReusedExchange [Reuses operator id: 11] Output [1]: [d_date_sk#110] -(54) CometBroadcastHashJoin +(56) 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 +(57) 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) CometScan parquet spark_catalog.default.web_site +(58) CometScan 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 +(59) CometFilter Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(58) CometBroadcastExchange +(60) CometBroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: [web_site_sk#111, web_site_id#112] -(59) CometBroadcastHashJoin +(61) 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 +(62) 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] +(63) CometColumnarToRow [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] +(64) 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 +(65) CometColumnarExchange 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] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(64) HashAggregate [codegen id : 6] +(66) CometColumnarToRow [codegen id : 6] +Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] + +(67) 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 +(68) Union -(66) HashAggregate [codegen id : 7] +(69) 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 +(70) 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, [plan_id=4] +Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(68) HashAggregate [codegen id : 8] +(71) CometColumnarToRow [codegen id : 8] +Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] + +(72) 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] +(73) ReusedExchange [Reuses operator id: 70] Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -(70) HashAggregate [codegen id : 16] +(74) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(75) 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] +(76) 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 +(77) CometColumnarExchange +Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] +Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(78) CometColumnarToRow [codegen id : 17] Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(73) HashAggregate [codegen id : 17] +(79) 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] +(80) ReusedExchange [Reuses operator id: 70] Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -(75) HashAggregate [codegen id : 25] +(81) CometColumnarToRow [codegen id : 25] +Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] + +(82) 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] +(83) 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 +(84) CometColumnarExchange Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(78) HashAggregate [codegen id : 26] +(85) CometColumnarToRow [codegen id : 26] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] + +(86) 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 +(87) Union -(80) HashAggregate [codegen id : 27] +(88) 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 +(89) 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] +(90) 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 +(91) 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] + +(92) CometColumnarToRow [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 (97) ++- * CometColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan parquet spark_catalog.default.date_dim (93) -(84) CometScan parquet spark_catalog.default.date_dim +(93) CometScan 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 +(94) 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 +(95) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(96) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(97) 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 = 24 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 = 27 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 = 44 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 = 47 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 c3b00538b8..bfb15e3f83 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,133 @@ -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) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow 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] - 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow 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] - 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] - 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 + 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 8227cd53db..f01361444d 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,46 @@ == 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 (42) ++- * Filter (41) + +- * HashAggregate (40) + +- * CometColumnarToRow (39) + +- CometColumnarExchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * CometColumnarToRow (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 (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * CometColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- BroadcastExchange (31) + +- * Filter (30) + +- * HashAggregate (29) + +- * CometColumnarToRow (28) + +- CometColumnarExchange (27) + +- * HashAggregate (26) + +- * CometColumnarToRow (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.item (23) (1) CometScan parquet spark_catalog.default.customer_address @@ -129,7 +131,7 @@ 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] +(19) CometColumnarToRow [codegen id : 4] Input [2]: [ca_state#2, ss_item_sk#5] (20) CometScan parquet spark_catalog.default.item @@ -143,7 +145,7 @@ ReadSchema: struct 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] +(22) CometColumnarToRow [codegen id : 3] Input [3]: [i_item_sk#13, i_current_price#14, i_category#15] (23) CometScan parquet spark_catalog.default.item @@ -157,7 +159,7 @@ ReadSchema: struct Input [2]: [i_current_price#16, i_category#17] Condition : isnotnull(i_category#17) -(25) ColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 1] Input [2]: [i_current_price#16, i_category#17] (26) HashAggregate [codegen id : 1] @@ -167,149 +169,155 @@ 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) CometColumnarExchange Input [3]: [i_category#17, sum#20, count#21] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(28) HashAggregate [codegen id : 2] +(28) CometColumnarToRow [codegen id : 2] +Input [3]: [i_category#17, sum#20, count#21] + +(29) 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] +(30) Filter [codegen id : 2] Input [2]: [avg(i_current_price)#23, i_category#17] Condition : isnotnull(avg(i_current_price)#23) -(30) BroadcastExchange +(31) 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] +(32) 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] +(33) 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 +(34) 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] +(35) 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] +(36) 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] +(37) 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 +(38) CometColumnarExchange +Input [2]: [ca_state#2, count#25] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] + +(39) CometColumnarToRow [codegen id : 5] Input [2]: [ca_state#2, count#25] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] -(38) HashAggregate [codegen id : 5] +(40) 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] +(41) Filter [codegen id : 5] Input [2]: [state#27, cnt#28] Condition : (cnt#28 >= 10) -(40) TakeOrderedAndProject +(42) 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 (47) ++- * CometColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(41) CometScan parquet spark_catalog.default.date_dim +(43) CometScan 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 +(44) 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 +(45) 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] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(45) BroadcastExchange +(47) 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 = 44 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan parquet spark_catalog.default.date_dim (46) +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* CometColumnarToRow (54) ++- CometHashAggregate (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) -(46) CometScan parquet spark_catalog.default.date_dim +(48) CometScan 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 +(49) 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 +(50) CometProject Input [3]: [d_month_seq#29, d_year#30, d_moy#31] Arguments: [d_month_seq#29], [d_month_seq#29] -(49) CometHashAggregate +(51) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(50) CometExchange +(52) CometExchange Input [1]: [d_month_seq#29] Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(51) CometHashAggregate +(53) CometHashAggregate Input [1]: [d_month_seq#29] Keys [1]: [d_month_seq#29] Functions: [] -(52) ColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [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 c5bbfc7d58..6f8c9456a2 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,72 +2,74 @@ 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 (1) - ColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [ca_state] #1 + WholeStageCodegen (4) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + CometColumnarToRow + 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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometExchange [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 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Project [i_item_sk] - BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] - ColumnarToRow + 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)] + CometColumnarToRow + 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 - 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #9 + WholeStageCodegen (1) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + CometColumnarToRow + 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 37e2319107..0e2aa161f1 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,46 +1,46 @@ == Physical Plan == -* ColumnarToRow (181) -+- CometSort (180) - +- CometColumnarExchange (179) - +- * Project (178) - +- * SortMergeJoin Inner (177) - :- * Sort (109) - : +- Exchange (108) - : +- * HashAggregate (107) - : +- * HashAggregate (106) - : +- * Project (105) - : +- * BroadcastHashJoin Inner BuildRight (104) - : :- * Project (98) - : : +- * BroadcastHashJoin Inner BuildRight (97) - : : :- * Project (95) - : : : +- * BroadcastHashJoin Inner BuildRight (94) - : : : :- * Project (89) - : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : :- * Project (86) - : : : : : +- * BroadcastHashJoin Inner BuildRight (85) - : : : : : :- * Project (80) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (79) - : : : : : : :- * Project (77) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (76) - : : : : : : : :- * Project (71) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (70) - : : : : : : : : :- * Project (65) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (64) - : : : : : : : : : :- * Project (62) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (61) - : : : : : : : : : : :- * Project (56) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : : : : : : : : :- * Project (53) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : : : : : : : : :- * Project (47) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : : : : : : : : : : :- * Project (41) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * ColumnarToRow (11) +* CometColumnarToRow (183) ++- CometSort (182) + +- CometColumnarExchange (181) + +- CometProject (180) + +- CometSortMergeJoin (179) + :- CometSort (110) + : +- CometColumnarExchange (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) + : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) : : : : : : : : : : : : : : : : : +- CometSort (10) : : : : : : : : : : : : : : : : : +- CometExchange (9) : : : : : : : : : : : : : : : : : +- CometProject (8) @@ -51,135 +51,137 @@ : : : : : : : : : : : : : : : : : +- CometProject (6) : : : : : : : : : : : : : : : : : +- CometFilter (5) : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- Exchange (26) - : : : : : : : : : : : : : : : : +- * HashAggregate (25) - : : : : : : : : : : : : : : : : +- * ColumnarToRow (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (39) - : : : : : : : : : : : : : : +- * ColumnarToRow (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (45) - : : : : : : : : : : : : : +- * ColumnarToRow (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- BroadcastExchange (51) - : : : : : : : : : : : : +- * ColumnarToRow (50) - : : : : : : : : : : : : +- CometFilter (49) - : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (48) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- BroadcastExchange (60) - : : : : : : : : : : +- * ColumnarToRow (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- BroadcastExchange (69) - : : : : : : : : +- * ColumnarToRow (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- BroadcastExchange (75) - : : : : : : : +- * ColumnarToRow (74) - : : : : : : : +- CometFilter (73) - : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (72) - : : : : : : +- ReusedExchange (78) - : : : : : +- BroadcastExchange (84) - : : : : : +- * ColumnarToRow (83) - : : : : : +- CometFilter (82) - : : : : : +- CometScan parquet spark_catalog.default.customer_address (81) - : : : : +- ReusedExchange (87) - : : : +- BroadcastExchange (93) - : : : +- * ColumnarToRow (92) - : : : +- CometFilter (91) - : : : +- CometScan parquet spark_catalog.default.income_band (90) - : : +- ReusedExchange (96) - : +- BroadcastExchange (103) - : +- * ColumnarToRow (102) - : +- CometProject (101) - : +- CometFilter (100) - : +- CometScan parquet spark_catalog.default.item (99) - +- * Sort (176) - +- Exchange (175) - +- * HashAggregate (174) - +- * HashAggregate (173) - +- * Project (172) - +- * BroadcastHashJoin Inner BuildRight (171) - :- * Project (169) - : +- * BroadcastHashJoin Inner BuildRight (168) - : :- * Project (166) - : : +- * BroadcastHashJoin Inner BuildRight (165) - : : :- * Project (163) - : : : +- * BroadcastHashJoin Inner BuildRight (162) - : : : :- * Project (160) - : : : : +- * BroadcastHashJoin Inner BuildRight (159) - : : : : :- * Project (157) - : : : : : +- * BroadcastHashJoin Inner BuildRight (156) - : : : : : :- * Project (154) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (153) - : : : : : : :- * Project (151) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (150) - : : : : : : : :- * Project (148) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (147) - : : : : : : : : :- * Project (145) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (144) - : : : : : : : : : :- * Project (142) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (141) - : : : : : : : : : : :- * Project (139) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (138) - : : : : : : : : : : : :- * Project (136) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (135) - : : : : : : : : : : : : :- * Project (133) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (132) - : : : : : : : : : : : : : :- * Project (130) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (129) - : : : : : : : : : : : : : : :- * Project (127) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (126) - : : : : : : : : : : : : : : : :- * ColumnarToRow (120) - : : : : : : : : : : : : : : : : +- CometSort (119) - : : : : : : : : : : : : : : : : +- CometExchange (118) - : : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (116) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (112) - : : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (110) - : : : : : : : : : : : : : : : : +- CometProject (115) - : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (113) - : : : : : : : : : : : : : : : +- * Sort (125) - : : : : : : : : : : : : : : : +- * Project (124) - : : : : : : : : : : : : : : : +- * Filter (123) - : : : : : : : : : : : : : : : +- * HashAggregate (122) - : : : : : : : : : : : : : : : +- ReusedExchange (121) - : : : : : : : : : : : : : : +- ReusedExchange (128) - : : : : : : : : : : : : : +- ReusedExchange (131) - : : : : : : : : : : : : +- ReusedExchange (134) - : : : : : : : : : : : +- ReusedExchange (137) - : : : : : : : : : : +- ReusedExchange (140) - : : : : : : : : : +- ReusedExchange (143) - : : : : : : : : +- ReusedExchange (146) - : : : : : : : +- ReusedExchange (149) - : : : : : : +- ReusedExchange (152) - : : : : : +- ReusedExchange (155) - : : : : +- ReusedExchange (158) - : : : +- ReusedExchange (161) - : : +- ReusedExchange (164) - : +- ReusedExchange (167) - +- ReusedExchange (170) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (27) + : : : : : : : : : : : : : : : : +- CometColumnarExchange (26) + : : : : : : : : : : : : : : : : +- * HashAggregate (25) + : : : : : : : : : : : : : : : : +- * CometColumnarToRow (24) + : : : : : : : : : : : : : : : : +- CometProject (23) + : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) + : : : : : : : : : : : : : : : : :- CometSort (16) + : : : : : : : : : : : : : : : : : +- CometExchange (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- CometSort (21) + : : : : : : : : : : : : : : : : +- CometExchange (20) + : : : : : : : : : : : : : : : : +- CometProject (19) + : : : : : : : : : : : : : : : : +- CometFilter (18) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * CometColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * CometColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * CometColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * CometColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * CometColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * CometColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * CometColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * CometColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * CometColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- CometSort (178) + +- CometColumnarExchange (177) + +- * HashAggregate (176) + +- * HashAggregate (175) + +- * 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) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (131) + : : : : : : : : : : : : : : :- * Project (129) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (128) + : : : : : : : : : : : : : : : :- * CometColumnarToRow (121) + : : : : : : : : : : : : : : : : +- CometSort (120) + : : : : : : : : : : : : : : : : +- CometExchange (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 (127) + : : : : : : : : : : : : : : : +- * Project (126) + : : : : : : : : : : : : : : : +- * Filter (125) + : : : : : : : : : : : : : : : +- * HashAggregate (124) + : : : : : : : : : : : : : : : +- * CometColumnarToRow (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (130) + : : : : : : : : : : : : : +- 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) (1) CometScan parquet spark_catalog.default.store_sales @@ -230,7 +232,7 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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, 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) ColumnarToRow [codegen id : 1] +(11) CometColumnarToRow [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] (12) CometScan parquet spark_catalog.default.catalog_sales @@ -288,7 +290,7 @@ Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number# 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] +(24) CometColumnarToRow [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] (25) HashAggregate [codegen id : 2] @@ -298,396 +300,399 @@ Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum((( Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] -(26) Exchange +(26) 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] -(27) HashAggregate [codegen id : 3] +(27) CometColumnarToRow [codegen id : 3] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(28) 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] -(28) Filter [codegen id : 3] +(29) 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))) -(29) Project [codegen id : 3] +(30) Project [codegen id : 3] Output [1]: [cs_item_sk#17] Input [3]: [cs_item_sk#17, sale#35, refund#36] -(30) Sort [codegen id : 3] +(31) Sort [codegen id : 3] Input [1]: [cs_item_sk#17] Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 19] +(32) SortMergeJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [cs_item_sk#17] Join type: Inner Join condition: None -(32) Project [codegen id : 19] +(33) 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] -(33) ReusedExchange [Reuses operator id: 185] +(34) ReusedExchange [Reuses operator id: 187] Output [2]: [d_date_sk#37, d_year#38] -(34) BroadcastHashJoin [codegen id : 19] +(35) 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 -(35) Project [codegen id : 19] +(36) 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] -(36) CometScan parquet spark_catalog.default.store +(37) CometScan 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 -(37) CometFilter +(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)) -(38) ColumnarToRow [codegen id : 5] +(39) CometColumnarToRow [codegen id : 5] Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] -(39) BroadcastExchange +(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] -(40) BroadcastHashJoin [codegen id : 19] +(41) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_store_sk#6] Right keys [1]: [s_store_sk#39] Join type: Inner Join condition: None -(41) Project [codegen id : 19] +(42) 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] -(42) CometScan parquet spark_catalog.default.customer +(43) CometScan 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 -(43) CometFilter +(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)) -(44) ColumnarToRow [codegen id : 6] +(45) CometColumnarToRow [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] -(45) BroadcastExchange +(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] -(46) BroadcastHashJoin [codegen id : 19] +(47) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_customer_sk#2] Right keys [1]: [c_customer_sk#42] Join type: Inner Join condition: None -(47) Project [codegen id : 19] +(48) 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] -(48) CometScan parquet spark_catalog.default.date_dim +(49) CometScan 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 -(49) CometFilter +(50) CometFilter Input [2]: [d_date_sk#48, d_year#49] Condition : isnotnull(d_date_sk#48) -(50) ColumnarToRow [codegen id : 7] +(51) CometColumnarToRow [codegen id : 7] Input [2]: [d_date_sk#48, d_year#49] -(51) BroadcastExchange +(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] -(52) BroadcastHashJoin [codegen id : 19] +(53) 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 -(53) Project [codegen id : 19] +(54) 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] -(54) ReusedExchange [Reuses operator id: 51] +(55) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#50, d_year#51] -(55) BroadcastHashJoin [codegen id : 19] +(56) 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 -(56) Project [codegen id : 19] +(57) 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] -(57) CometScan parquet spark_catalog.default.customer_demographics +(58) CometScan 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 -(58) CometFilter +(59) CometFilter Input [2]: [cd_demo_sk#52, cd_marital_status#53] Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) -(59) ColumnarToRow [codegen id : 9] +(60) CometColumnarToRow [codegen id : 9] Input [2]: [cd_demo_sk#52, cd_marital_status#53] -(60) BroadcastExchange +(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] -(61) BroadcastHashJoin [codegen id : 19] +(62) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_cdemo_sk#3] Right keys [1]: [cd_demo_sk#52] Join type: Inner Join condition: None -(62) Project [codegen id : 19] +(63) 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] -(63) ReusedExchange [Reuses operator id: 60] +(64) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#54, cd_marital_status#55] -(64) BroadcastHashJoin [codegen id : 19] +(65) 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) -(65) Project [codegen id : 19] +(66) 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] -(66) CometScan parquet spark_catalog.default.promotion +(67) CometScan 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 -(67) CometFilter +(68) CometFilter Input [1]: [p_promo_sk#56] Condition : isnotnull(p_promo_sk#56) -(68) ColumnarToRow [codegen id : 11] +(69) CometColumnarToRow [codegen id : 11] Input [1]: [p_promo_sk#56] -(69) BroadcastExchange +(70) BroadcastExchange Input [1]: [p_promo_sk#56] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -(70) BroadcastHashJoin [codegen id : 19] +(71) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_promo_sk#7] Right keys [1]: [p_promo_sk#56] Join type: Inner Join condition: None -(71) Project [codegen id : 19] +(72) 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] -(72) CometScan parquet spark_catalog.default.household_demographics +(73) CometScan 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 -(73) CometFilter +(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)) -(74) ColumnarToRow [codegen id : 12] +(75) CometColumnarToRow [codegen id : 12] Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] -(75) BroadcastExchange +(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] -(76) BroadcastHashJoin [codegen id : 19] +(77) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_hdemo_sk#4] Right keys [1]: [hd_demo_sk#57] Join type: Inner Join condition: None -(77) Project [codegen id : 19] +(78) 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] -(78) ReusedExchange [Reuses operator id: 75] +(79) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] -(79) BroadcastHashJoin [codegen id : 19] +(80) 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 -(80) Project [codegen id : 19] +(81) 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] -(81) CometScan parquet spark_catalog.default.customer_address +(82) CometScan 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 -(82) CometFilter +(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) -(83) ColumnarToRow [codegen id : 14] +(84) CometColumnarToRow [codegen id : 14] Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] -(84) BroadcastExchange +(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] -(85) BroadcastHashJoin [codegen id : 19] +(86) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_addr_sk#5] Right keys [1]: [ca_address_sk#61] Join type: Inner Join condition: None -(86) Project [codegen id : 19] +(87) 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] -(87) ReusedExchange [Reuses operator id: 84] +(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] -(88) BroadcastHashJoin [codegen id : 19] +(89) 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 -(89) Project [codegen id : 19] +(90) 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] -(90) CometScan parquet spark_catalog.default.income_band +(91) CometScan 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 -(91) CometFilter +(92) CometFilter Input [1]: [ib_income_band_sk#71] Condition : isnotnull(ib_income_band_sk#71) -(92) ColumnarToRow [codegen id : 16] +(93) CometColumnarToRow [codegen id : 16] Input [1]: [ib_income_band_sk#71] -(93) BroadcastExchange +(94) BroadcastExchange Input [1]: [ib_income_band_sk#71] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] -(94) BroadcastHashJoin [codegen id : 19] +(95) 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 -(95) Project [codegen id : 19] +(96) 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] -(96) ReusedExchange [Reuses operator id: 93] +(97) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#72] -(97) BroadcastHashJoin [codegen id : 19] +(98) 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 -(98) Project [codegen id : 19] +(99) 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] -(99) CometScan parquet spark_catalog.default.item +(100) CometScan 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 -(100) CometFilter +(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)) -(101) CometProject +(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] -(102) ColumnarToRow [codegen id : 18] +(103) CometColumnarToRow [codegen id : 18] Input [2]: [i_item_sk#73, i_product_name#76] -(103) BroadcastExchange +(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] -(104) BroadcastHashJoin [codegen id : 19] +(105) BroadcastHashJoin [codegen id : 19] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#73] Join type: Inner Join condition: None -(105) Project [codegen id : 19] +(106) 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] -(106) HashAggregate [codegen id : 19] +(107) 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] -(107) HashAggregate [codegen id : 19] +(108) 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] -(108) Exchange +(109) 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] -(109) Sort [codegen id : 20] +(110) 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] -(110) CometScan parquet spark_catalog.default.store_sales +(111) CometScan 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 [] @@ -695,368 +700,370 @@ 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 -(111) CometFilter +(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)) -(112) CometBroadcastExchange +(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] -(113) CometScan parquet spark_catalog.default.store_returns +(114) CometScan 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 -(114) CometFilter +(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)) -(115) CometProject +(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] -(116) CometBroadcastHashJoin +(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 -(117) CometProject +(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] -(118) CometExchange +(119) CometExchange 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, CometNativeShuffle, [plan_id=15] -(119) CometSort +(120) 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: [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] -(120) ColumnarToRow [codegen id : 21] +(121) CometColumnarToRow [codegen id : 20] 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] -(121) ReusedExchange [Reuses operator id: 26] +(122) ReusedExchange [Reuses operator id: 26] Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] -(122) HashAggregate [codegen id : 23] +(123) CometColumnarToRow [codegen id : 22] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(124) HashAggregate [codegen id : 22] 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] -(123) Filter [codegen id : 23] +(125) Filter [codegen id : 22] 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))) -(124) Project [codegen id : 23] +(126) Project [codegen id : 22] Output [1]: [cs_item_sk#122] Input [3]: [cs_item_sk#122, sale#130, refund#131] -(125) Sort [codegen id : 23] +(127) Sort [codegen id : 22] Input [1]: [cs_item_sk#122] Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 -(126) SortMergeJoin [codegen id : 39] +(128) SortMergeJoin [codegen id : 38] Left keys [1]: [ss_item_sk#106] Right keys [1]: [cs_item_sk#122] Join type: Inner Join condition: None -(127) Project [codegen id : 39] +(129) Project [codegen id : 38] 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] -(128) ReusedExchange [Reuses operator id: 189] +(130) ReusedExchange [Reuses operator id: 191] Output [2]: [d_date_sk#132, d_year#133] -(129) BroadcastHashJoin [codegen id : 39] +(131) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_sold_date_sk#117] Right keys [1]: [d_date_sk#132] Join type: Inner Join condition: None -(130) Project [codegen id : 39] +(132) Project [codegen id : 38] 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] -(131) ReusedExchange [Reuses operator id: 39] +(133) ReusedExchange [Reuses operator id: 40] Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] -(132) BroadcastHashJoin [codegen id : 39] +(134) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_store_sk#111] Right keys [1]: [s_store_sk#134] Join type: Inner Join condition: None -(133) Project [codegen id : 39] +(135) Project [codegen id : 38] 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] -(134) ReusedExchange [Reuses operator id: 45] +(136) ReusedExchange [Reuses operator id: 46] 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] -(135) BroadcastHashJoin [codegen id : 39] +(137) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_customer_sk#107] Right keys [1]: [c_customer_sk#137] Join type: Inner Join condition: None -(136) Project [codegen id : 39] +(138) Project [codegen id : 38] 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] -(137) ReusedExchange [Reuses operator id: 51] +(139) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#143, d_year#144] -(138) BroadcastHashJoin [codegen id : 39] +(140) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_first_sales_date_sk#142] Right keys [1]: [d_date_sk#143] Join type: Inner Join condition: None -(139) Project [codegen id : 39] +(141) Project [codegen id : 38] 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] -(140) ReusedExchange [Reuses operator id: 51] +(142) ReusedExchange [Reuses operator id: 52] Output [2]: [d_date_sk#145, d_year#146] -(141) BroadcastHashJoin [codegen id : 39] +(143) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_first_shipto_date_sk#141] Right keys [1]: [d_date_sk#145] Join type: Inner Join condition: None -(142) Project [codegen id : 39] +(144) Project [codegen id : 38] 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] -(143) ReusedExchange [Reuses operator id: 60] +(145) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#147, cd_marital_status#148] -(144) BroadcastHashJoin [codegen id : 39] +(146) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_cdemo_sk#108] Right keys [1]: [cd_demo_sk#147] Join type: Inner Join condition: None -(145) Project [codegen id : 39] +(147) Project [codegen id : 38] 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] -(146) ReusedExchange [Reuses operator id: 60] +(148) ReusedExchange [Reuses operator id: 61] Output [2]: [cd_demo_sk#149, cd_marital_status#150] -(147) BroadcastHashJoin [codegen id : 39] +(149) BroadcastHashJoin [codegen id : 38] 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) -(148) Project [codegen id : 39] +(150) Project [codegen id : 38] 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] -(149) ReusedExchange [Reuses operator id: 69] +(151) ReusedExchange [Reuses operator id: 70] Output [1]: [p_promo_sk#151] -(150) BroadcastHashJoin [codegen id : 39] +(152) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_promo_sk#112] Right keys [1]: [p_promo_sk#151] Join type: Inner Join condition: None -(151) Project [codegen id : 39] +(153) Project [codegen id : 38] 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] -(152) ReusedExchange [Reuses operator id: 75] +(154) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] -(153) BroadcastHashJoin [codegen id : 39] +(155) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_hdemo_sk#109] Right keys [1]: [hd_demo_sk#152] Join type: Inner Join condition: None -(154) Project [codegen id : 39] +(156) Project [codegen id : 38] 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] -(155) ReusedExchange [Reuses operator id: 75] +(157) ReusedExchange [Reuses operator id: 76] Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] -(156) BroadcastHashJoin [codegen id : 39] +(158) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_current_hdemo_sk#139] Right keys [1]: [hd_demo_sk#154] Join type: Inner Join condition: None -(157) Project [codegen id : 39] +(159) Project [codegen id : 38] 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] -(158) ReusedExchange [Reuses operator id: 84] +(160) ReusedExchange [Reuses operator id: 85] Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] -(159) BroadcastHashJoin [codegen id : 39] +(161) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_addr_sk#110] Right keys [1]: [ca_address_sk#156] Join type: Inner Join condition: None -(160) Project [codegen id : 39] +(162) Project [codegen id : 38] 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] -(161) ReusedExchange [Reuses operator id: 84] +(163) ReusedExchange [Reuses operator id: 85] Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] -(162) BroadcastHashJoin [codegen id : 39] +(164) BroadcastHashJoin [codegen id : 38] Left keys [1]: [c_current_addr_sk#140] Right keys [1]: [ca_address_sk#161] Join type: Inner Join condition: None -(163) Project [codegen id : 39] +(165) Project [codegen id : 38] 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] -(164) ReusedExchange [Reuses operator id: 93] +(166) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#166] -(165) BroadcastHashJoin [codegen id : 39] +(167) BroadcastHashJoin [codegen id : 38] Left keys [1]: [hd_income_band_sk#153] Right keys [1]: [ib_income_band_sk#166] Join type: Inner Join condition: None -(166) Project [codegen id : 39] +(168) Project [codegen id : 38] 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] -(167) ReusedExchange [Reuses operator id: 93] +(169) ReusedExchange [Reuses operator id: 94] Output [1]: [ib_income_band_sk#167] -(168) BroadcastHashJoin [codegen id : 39] +(170) BroadcastHashJoin [codegen id : 38] Left keys [1]: [hd_income_band_sk#155] Right keys [1]: [ib_income_band_sk#167] Join type: Inner Join condition: None -(169) Project [codegen id : 39] +(171) Project [codegen id : 38] 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] -(170) ReusedExchange [Reuses operator id: 103] +(172) ReusedExchange [Reuses operator id: 104] Output [2]: [i_item_sk#168, i_product_name#169] -(171) BroadcastHashJoin [codegen id : 39] +(173) BroadcastHashJoin [codegen id : 38] Left keys [1]: [ss_item_sk#106] Right keys [1]: [i_item_sk#168] Join type: Inner Join condition: None -(172) Project [codegen id : 39] +(174) Project [codegen id : 38] 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] -(173) HashAggregate [codegen id : 39] +(175) HashAggregate [codegen id : 38] 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] -(174) HashAggregate [codegen id : 39] +(176) HashAggregate [codegen id : 38] 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] -(175) Exchange +(177) 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, [plan_id=16] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] -(176) Sort [codegen id : 40] +(178) CometSort 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 +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) 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) CometSortMergeJoin +Left output [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] +Right output [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: [item_sk#90, store_name#91, store_zip#92], [item_sk#176, store_name#177, store_zip#178], Inner, (cnt#180 <= cnt#102) -(178) 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] +(180) CometProject 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] +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, 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] -(179) CometColumnarExchange +(181) 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] -(180) CometSort +(182) 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: [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) ColumnarToRow [codegen id : 42] +(183) CometColumnarToRow [codegen id : 39] 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] ===== 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 (187) ++- * CometColumnarToRow (186) + +- CometFilter (185) + +- CometScan parquet spark_catalog.default.date_dim (184) -(182) CometScan parquet spark_catalog.default.date_dim +(184) CometScan 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 +(185) 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] +(186) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#37, d_year#38] -(185) BroadcastExchange +(187) 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 = 110 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 = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (191) ++- * CometColumnarToRow (190) + +- CometFilter (189) + +- CometScan parquet spark_catalog.default.date_dim (188) -(186) CometScan parquet spark_catalog.default.date_dim +(188) CometScan 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 +(189) 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] +(190) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#132, d_year#133] -(189) BroadcastExchange +(191) 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 f43131af19..ddb0c9b82e 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,268 +1,263 @@ -WholeStageCodegen (42) - ColumnarToRow +WholeStageCodegen (39) + CometColumnarToRow InputAdapter 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 - 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) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #2 - 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 - 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] - CometExchange [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] - InputAdapter - Exchange [cs_item_sk] #6 - 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 + CometProject [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] + CometSortMergeJoin [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,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + 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 + 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) + CometColumnarToRow + InputAdapter + 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] + CometExchange [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) + CometColumnarToRow InputAdapter - 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] - CometExchange [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] - CometExchange [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 + 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 - 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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_item_sk] #6 + 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] + CometColumnarToRow + InputAdapter + 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] + CometExchange [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] + CometExchange [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 - 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] + ReusedExchange [d_date_sk,d_year] #5 InputAdapter - BroadcastExchange #11 - WholeStageCodegen (7) - ColumnarToRow + BroadcastExchange #9 + WholeStageCodegen (5) + CometColumnarToRow InputAdapter - CometFilter [d_date_sk,d_year] - CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + 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] #11 + BroadcastExchange #10 + WholeStageCodegen (6) + CometColumnarToRow + 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 #12 - WholeStageCodegen (9) - ColumnarToRow + BroadcastExchange #11 + WholeStageCodegen (7) + CometColumnarToRow InputAdapter - CometFilter [cd_demo_sk,cd_marital_status] - CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + CometFilter [d_date_sk,d_year] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - BroadcastExchange #13 - WholeStageCodegen (11) - ColumnarToRow + BroadcastExchange #12 + WholeStageCodegen (9) + CometColumnarToRow InputAdapter - CometFilter [p_promo_sk] - CometScan parquet spark_catalog.default.promotion [p_promo_sk] + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + BroadcastExchange #13 + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] InputAdapter - BroadcastExchange #15 - WholeStageCodegen (14) - ColumnarToRow + BroadcastExchange #14 + WholeStageCodegen (12) + CometColumnarToRow 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 [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - ColumnarToRow + BroadcastExchange #15 + WholeStageCodegen (14) + CometColumnarToRow InputAdapter - CometFilter [ib_income_band_sk] - CometScan parquet spark_catalog.default.income_band [ib_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 [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - BroadcastExchange #17 - WholeStageCodegen (18) - ColumnarToRow + BroadcastExchange #16 + WholeStageCodegen (16) + CometColumnarToRow 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 (40) - Sort [item_sk,store_name,store_zip] - InputAdapter - Exchange [item_sk,store_name,store_zip] #18 - 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 - 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] - CometExchange [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] - InputAdapter - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #21 + 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 (18) + CometColumnarToRow + 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] + CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] + CometColumnarExchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (38) + 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 - ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + WholeStageCodegen (20) + CometColumnarToRow + InputAdapter + 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] + CometExchange [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) + CometColumnarToRow + 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 (22) + 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] + CometColumnarToRow + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 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] #21 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 InputAdapter - ReusedExchange [d_date_sk,d_year] #11 + 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 [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [cd_demo_sk,cd_marital_status] #12 + ReusedExchange [d_date_sk,d_year] #11 InputAdapter - ReusedExchange [p_promo_sk] #13 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [cd_demo_sk,cd_marital_status] #12 InputAdapter - ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + ReusedExchange [p_promo_sk] #13 InputAdapter - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + 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 + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [ib_income_band_sk] #16 + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 InputAdapter - ReusedExchange [i_item_sk,i_product_name] #17 + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + 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 49a984b30c..032864d469 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,93 @@ == 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 (89) ++- * Filter (88) + +- Window (87) + +- WindowGroupLimit (86) + +- * CometColumnarToRow (85) + +- CometSort (84) + +- CometColumnarExchange (83) + +- WindowGroupLimit (82) + +- * Sort (81) + +- Union (80) + :- * HashAggregate (23) + : +- * CometColumnarToRow (22) + : +- CometColumnarExchange (21) + : +- * HashAggregate (20) + : +- * CometColumnarToRow (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 (30) + : +- * CometColumnarToRow (29) + : +- CometColumnarExchange (28) + : +- * HashAggregate (27) + : +- * HashAggregate (26) + : +- * CometColumnarToRow (25) + : +- ReusedExchange (24) + :- * HashAggregate (37) + : +- * CometColumnarToRow (36) + : +- CometColumnarExchange (35) + : +- * HashAggregate (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + :- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- CometColumnarExchange (42) + : +- * HashAggregate (41) + : +- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- ReusedExchange (38) + :- * HashAggregate (51) + : +- * CometColumnarToRow (50) + : +- CometColumnarExchange (49) + : +- * HashAggregate (48) + : +- * HashAggregate (47) + : +- * CometColumnarToRow (46) + : +- ReusedExchange (45) + :- * HashAggregate (58) + : +- * CometColumnarToRow (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * HashAggregate (54) + : +- * CometColumnarToRow (53) + : +- ReusedExchange (52) + :- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * HashAggregate (61) + : +- * CometColumnarToRow (60) + : +- ReusedExchange (59) + :- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- CometColumnarExchange (70) + : +- * HashAggregate (69) + : +- * HashAggregate (68) + : +- * CometColumnarToRow (67) + : +- ReusedExchange (66) + +- * HashAggregate (79) + +- * CometColumnarToRow (78) + +- CometColumnarExchange (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- * CometColumnarToRow (74) + +- ReusedExchange (73) (1) CometScan parquet spark_catalog.default.store_sales @@ -160,7 +178,7 @@ Arguments: [ss_item_sk#1], [i_item_sk#14], Inner, BuildRight Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] -(19) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [codegen id : 1] Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#13, i_brand#15, i_class#16, i_category#17, i_product_name#18] (20) HashAggregate [codegen id : 1] @@ -170,304 +188,358 @@ 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) 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, [plan_id=1] +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] -(22) HashAggregate [codegen id : 2] +(22) CometColumnarToRow [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] + +(23) 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] +(24) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, s_store_id#40, sum#41, isEmpty#42] -(24) HashAggregate [codegen id : 4] +(25) CometColumnarToRow [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] + +(26) 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] +(27) 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 +(28) 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, CometColumnarShuffle, [plan_id=2] + +(29) CometColumnarToRow [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] -Arguments: hashpartitioning(i_category#33, i_class#34, i_brand#35, i_product_name#36, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, [plan_id=2] -(27) HashAggregate [codegen id : 5] +(30) 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] +(31) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#53, i_class#54, i_brand#55, i_product_name#56, d_year#57, d_qoy#58, d_moy#59, s_store_id#60, sum#61, isEmpty#62] -(29) HashAggregate [codegen id : 7] +(32) CometColumnarToRow [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] + +(33) 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] +(34) 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 +(35) 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] + +(36) CometColumnarToRow [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] -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] +(37) 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] +(38) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] -(34) HashAggregate [codegen id : 10] +(39) CometColumnarToRow [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] + +(40) 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] +(41) 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 +(42) 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] + +(43) CometColumnarToRow [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] +(44) 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] +(45) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] -(39) HashAggregate [codegen id : 13] +(46) CometColumnarToRow [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] + +(47) 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] +(48) 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 +(49) 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, CometColumnarShuffle, [plan_id=5] + +(50) CometColumnarToRow [codegen id : 14] Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] -Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(42) HashAggregate [codegen id : 14] +(51) 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] +(52) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#119, i_class#120, i_brand#121, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sum#127, isEmpty#128] -(44) HashAggregate [codegen id : 16] +(53) CometColumnarToRow [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] + +(54) 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] +(55) 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 +(56) 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] +(57) CometColumnarToRow [codegen id : 17] +Input [5]: [i_category#119, i_class#120, i_brand#121, sum#134, isEmpty#135] + +(58) 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] +(59) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sum#151, isEmpty#152] -(49) HashAggregate [codegen id : 19] +(60) CometColumnarToRow [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] + +(61) 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] +(62) 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 +(63) 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] + +(64) CometColumnarToRow [codegen id : 20] 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] +(65) 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] +(66) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#168, i_class#169, i_brand#170, i_product_name#171, d_year#172, d_qoy#173, d_moy#174, s_store_id#175, sum#176, isEmpty#177] -(54) HashAggregate [codegen id : 22] +(67) CometColumnarToRow [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] + +(68) 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] +(69) 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 +(70) CometColumnarExchange Input [3]: [i_category#168, sum#183, isEmpty#184] -Arguments: hashpartitioning(i_category#168, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(i_category#168, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(57) HashAggregate [codegen id : 23] +(71) CometColumnarToRow [codegen id : 23] +Input [3]: [i_category#168, sum#183, isEmpty#184] + +(72) 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] +(73) ReusedExchange [Reuses operator id: 21] Output [10]: [i_category#194, i_class#195, i_brand#196, i_product_name#197, d_year#198, d_qoy#199, d_moy#200, s_store_id#201, sum#202, isEmpty#203] -(59) HashAggregate [codegen id : 25] +(74) CometColumnarToRow [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] + +(75) 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] +(76) 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 +(77) 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] +(78) CometColumnarToRow [codegen id : 26] +Input [2]: [sum#209, isEmpty#210] + +(79) 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 +(80) Union -(64) Sort [codegen id : 27] +(81) 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 +(82) 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 +(83) 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, [plan_id=10] +Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(67) Sort [codegen id : 28] +(84) 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] + +(85) CometColumnarToRow [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 +(86) 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 +(87) 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] +(88) 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 +(89) 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 (94) ++- * CometColumnarToRow (93) + +- CometProject (92) + +- CometFilter (91) + +- CometScan parquet spark_catalog.default.date_dim (90) -(72) CometScan parquet spark_catalog.default.date_dim +(90) CometScan 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 +(91) 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 +(92) 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] +(93) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -(76) BroadcastExchange +(94) 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 cc9627e89b..93a917f74d 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,135 @@ 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] + CometColumnarToRow 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + 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 ccef440881..a1a3eeec27 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,69 @@ == 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 (65) ++- * Project (64) + +- Window (63) + +- * CometColumnarToRow (62) + +- CometSort (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- * HashAggregate (57) + +- Union (56) + :- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * CometColumnarToRow (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 (35) + : +- * BroadcastHashJoin LeftSemi BuildRight (34) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (33) + : +- * Project (32) + : +- * Filter (31) + : +- Window (30) + : +- WindowGroupLimit (29) + : +- * Sort (28) + : +- * HashAggregate (27) + : +- * CometColumnarToRow (26) + : +- CometColumnarExchange (25) + : +- * HashAggregate (24) + : +- * CometColumnarToRow (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) + : +- * CometColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (55) + +- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- ReusedExchange (49) (1) CometScan parquet spark_catalog.default.store_sales @@ -99,7 +106,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) CometColumnarToRow [codegen id : 5] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -113,7 +120,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) CometColumnarToRow [codegen id : 4] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -164,7 +171,7 @@ Arguments: [ss_sold_date_sk#12], [d_date_sk#16], Inner, BuildRight Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#15, d_date_sk#16] Arguments: [ss_net_profit#11, s_state#15], [ss_net_profit#11, s_state#15] -(23) ColumnarToRow [codegen id : 1] +(23) CometColumnarToRow [codegen id : 1] Input [2]: [ss_net_profit#11, s_state#15] (24) HashAggregate [codegen id : 1] @@ -174,204 +181,223 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Aggregate Attributes [1]: [sum#17] Results [2]: [s_state#15, sum#18] -(25) Exchange +(25) CometColumnarExchange Input [2]: [s_state#15, sum#18] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(26) HashAggregate [codegen id : 2] +(26) CometColumnarToRow [codegen id : 2] +Input [2]: [s_state#15, sum#18] + +(27) 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] +(28) 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 +(29) 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 +(30) 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] +(31) 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] +(32) Project [codegen id : 3] Output [1]: [s_state#15] Input [4]: [s_state#15, _w0#20, s_state#15, ranking#21] -(32) BroadcastExchange +(33) BroadcastExchange Input [1]: [s_state#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(33) BroadcastHashJoin [codegen id : 4] +(34) BroadcastHashJoin [codegen id : 4] Left keys [1]: [s_state#9] Right keys [1]: [s_state#15] Join type: LeftSemi Join condition: None -(34) BroadcastExchange +(35) 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] +(36) 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] +(37) 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] +(38) 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 +(39) 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] + +(40) CometColumnarToRow [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] +(41) 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] +(42) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#29, s_county#30, sum#31] -(41) HashAggregate [codegen id : 12] +(43) CometColumnarToRow [codegen id : 12] +Input [3]: [s_state#29, s_county#30, sum#31] + +(44) 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] +(45) 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 +(46) CometColumnarExchange +Input [3]: [s_state#29, sum#36, isEmpty#37] +Arguments: hashpartitioning(s_state#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] + +(47) CometColumnarToRow [codegen id : 13] Input [3]: [s_state#29, sum#36, isEmpty#37] -Arguments: hashpartitioning(s_state#29, 5), ENSURE_REQUIREMENTS, [plan_id=5] -(44) HashAggregate [codegen id : 13] +(48) 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] +(49) ReusedExchange [Reuses operator id: 39] Output [3]: [s_state#44, s_county#45, sum#46] -(46) HashAggregate [codegen id : 19] +(50) CometColumnarToRow [codegen id : 19] +Input [3]: [s_state#44, s_county#45, sum#46] + +(51) 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] +(52) 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 +(53) CometColumnarExchange +Input [2]: [sum#51, isEmpty#52] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(54) CometColumnarToRow [codegen id : 20] Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(49) HashAggregate [codegen id : 20] +(55) 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 +(56) Union -(51) HashAggregate [codegen id : 21] +(57) 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 +(58) 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] +(59) 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 +(60) CometExchange +Input [5]: [total_sum#25, s_state#9, s_county#8, lochierarchy#28, _w0#60] +Arguments: hashpartitioning(lochierarchy#28, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(61) 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] +(62) CometColumnarToRow [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 +(63) 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] +(64) 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 +(65) 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 (70) ++- * CometColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) -(59) CometScan parquet spark_catalog.default.date_dim +(66) CometScan 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 +(67) 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 +(68) 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] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(63) BroadcastExchange +(70) 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 67f4841b4f..836b276832 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,29 +1,29 @@ 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) + CometColumnarToRow 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) - 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] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [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 + 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] + CometColumnarToRow InputAdapter - Exchange [s_state,s_county] #3 + CometColumnarExchange [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 + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (4) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -60,38 +60,43 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count 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 - WholeStageCodegen (13) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #8 + WholeStageCodegen (1) + HashAggregate [s_state,ss_net_profit] [sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange [s_state] #10 + CometColumnarExchange [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) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #11 + CometColumnarExchange #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 + CometColumnarToRow + 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 45f7e2e66d..4b97b311be 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 17b7fb0b81..33b2931019 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,79 @@ == 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 (75) ++- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- * CometColumnarToRow (16) + : : : +- CometColumnarExchange (15) + : : : +- * HashAggregate (14) + : : : +- * CometColumnarToRow (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 (36) + : : +- * HashAggregate (35) + : : +- * CometColumnarToRow (34) + : : +- CometColumnarExchange (33) + : : +- * HashAggregate (32) + : : +- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometProject (25) + : : : +- CometBroadcastHashJoin (24) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- CometBroadcastExchange (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.store_sales (21) + : : +- CometBroadcastExchange (28) + : : +- CometFilter (27) + : : +- CometScan parquet spark_catalog.default.date_dim (26) + : +- BroadcastExchange (54) + : +- * Filter (53) + : +- * HashAggregate (52) + : +- * CometColumnarToRow (51) + : +- CometColumnarExchange (50) + : +- * HashAggregate (49) + : +- * CometColumnarToRow (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometFilter (39) + : : : +- CometScan parquet spark_catalog.default.customer (38) + : : +- CometBroadcastExchange (42) + : : +- CometFilter (41) + : : +- CometScan parquet spark_catalog.default.web_sales (40) + : +- ReusedExchange (45) + +- BroadcastExchange (72) + +- * HashAggregate (71) + +- * CometColumnarToRow (70) + +- CometColumnarExchange (69) + +- * HashAggregate (68) + +- * CometColumnarToRow (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometProject (63) + : +- CometBroadcastHashJoin (62) + : :- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.customer (57) + : +- CometBroadcastExchange (61) + : +- CometFilter (60) + : +- CometScan parquet spark_catalog.default.web_sales (59) + +- ReusedExchange (64) (1) CometScan parquet spark_catalog.default.customer @@ -132,7 +136,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] Arguments: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10], [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] -(13) ColumnarToRow [codegen id : 1] +(13) CometColumnarToRow [codegen id : 1] Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] (14) HashAggregate [codegen id : 1] @@ -142,33 +146,36 @@ 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) 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, [plan_id=1] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(16) HashAggregate [codegen id : 8] +(16) CometColumnarToRow [codegen id : 8] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(17) 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] +(18) Filter [codegen id : 8] Input [2]: [customer_id#14, year_total#15] Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) -(18) CometScan parquet spark_catalog.default.customer +(19) CometScan 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 +(20) 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) CometScan parquet spark_catalog.default.store_sales +(21) CometScan 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 +183,93 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(21) CometFilter +(22) CometFilter Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] Condition : isnotnull(ss_customer_sk#20) -(22) CometBroadcastExchange +(23) 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 +(24) 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 +(25) 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) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(27) 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 +(28) CometBroadcastExchange Input [2]: [d_date_sk#24, d_year#25] Arguments: [d_date_sk#24, d_year#25] -(28) CometBroadcastHashJoin +(29) 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 +(30) 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] +(31) CometColumnarToRow [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] +(32) 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 +(33) 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] + +(34) CometColumnarToRow [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] +(35) 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 +(36) 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] +(37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [customer_id#14] Right keys [1]: [customer_id#28] Join type: Inner Join condition: None -(36) CometScan parquet spark_catalog.default.customer +(38) CometScan 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 +(39) 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) CometScan parquet spark_catalog.default.web_sales +(40) CometScan 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 +277,89 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(39) CometFilter +(41) 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 +(42) 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 +(43) 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 +(44) 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] +(45) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#40, d_year#41] -(44) CometBroadcastHashJoin +(46) 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 +(47) 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] +(48) CometColumnarToRow [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] +(49) 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 +(50) 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, [plan_id=4] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(49) HashAggregate [codegen id : 5] +(51) CometColumnarToRow [codegen id : 5] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(52) 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] +(53) 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 +(54) 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] +(55) 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] +(56) 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) CometScan parquet spark_catalog.default.customer +(57) CometScan 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 +(58) 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) CometScan parquet spark_catalog.default.web_sales +(59) CometScan 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 +367,131 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(57) CometFilter +(60) 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 +(61) 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 +(62) 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 +(63) 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] +(64) ReusedExchange [Reuses operator id: 28] Output [2]: [d_date_sk#55, d_year#56] -(62) CometBroadcastHashJoin +(65) 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 +(66) 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] +(67) CometColumnarToRow [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] +(68) 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 +(69) 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] + +(70) CometColumnarToRow [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] +(71) 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 +(72) 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] +(73) 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] +(74) 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 +(75) 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 (79) ++- * CometColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) -(72) CometScan parquet spark_catalog.default.date_dim +(76) CometScan 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 +(77) 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] +(78) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] -(75) BroadcastExchange +(79) 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 = 21 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometFilter (81) + +- CometScan parquet spark_catalog.default.date_dim (80) -(76) CometScan parquet spark_catalog.default.date_dim +(80) CometScan 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#24, d_year#25] -(79) BroadcastExchange +(83) 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 = 40 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 = 59 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 4360a95551..97849b3c9f 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,100 +7,104 @@ 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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) - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 + WholeStageCodegen (2) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + CometColumnarToRow + 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) + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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 109d01b5aa..f18a788ef9 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,129 +1,132 @@ == Physical Plan == -TakeOrderedAndProject (125) -+- * Project (124) - +- * SortMergeJoin Inner (123) - :- * Sort (67) - : +- Exchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- Exchange (63) - : +- * HashAggregate (62) - : +- * ColumnarToRow (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (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) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (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) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (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) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.web_returns (50) - +- * Sort (122) - +- Exchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- Exchange (118) - +- * HashAggregate (117) - +- * ColumnarToRow (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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) +* CometColumnarToRow (128) ++- CometTakeOrderedAndProject (127) + +- CometProject (126) + +- CometSortMergeJoin (125) + :- CometSort (68) + : +- CometColumnarExchange (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- * CometColumnarToRow (64) + : +- CometColumnarExchange (63) + : +- * HashAggregate (62) + : +- * CometColumnarToRow (61) + : +- CometHashAggregate (60) + : +- CometExchange (59) + : +- CometHashAggregate (58) + : +- CometUnion (57) + : :- CometProject (22) + : : +- CometSortMergeJoin (21) + : : :- CometSort (15) + : : : +- CometExchange (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) + : : +- CometExchange (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- CometProject (39) + : : +- CometSortMergeJoin (38) + : : :- CometSort (32) + : : : +- CometExchange (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) + : : +- CometExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.store_returns (33) + : +- CometProject (56) + : +- CometSortMergeJoin (55) + : :- CometSort (49) + : : +- CometExchange (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) + : +- CometExchange (53) + : +- CometProject (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.web_returns (50) + +- CometSort (124) + +- CometColumnarExchange (123) + +- * Filter (122) + +- * HashAggregate (121) + +- * CometColumnarToRow (120) + +- CometColumnarExchange (119) + +- * HashAggregate (118) + +- * CometColumnarToRow (117) + +- CometHashAggregate (116) + +- CometExchange (115) + +- CometHashAggregate (114) + +- CometUnion (113) + :- CometProject (84) + : +- CometSortMergeJoin (83) + : :- CometSort (80) + : : +- CometExchange (79) + : : +- CometProject (78) + : : +- CometBroadcastHashJoin (77) + : : :- CometProject (73) + : : : +- CometBroadcastHashJoin (72) + : : : :- CometFilter (70) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) + : : : +- ReusedExchange (71) + : : +- CometBroadcastExchange (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.date_dim (74) + : +- CometSort (82) + : +- ReusedExchange (81) + :- CometProject (98) + : +- CometSortMergeJoin (97) + : :- CometSort (94) + : : +- CometExchange (93) + : : +- CometProject (92) + : : +- CometBroadcastHashJoin (91) + : : :- CometProject (89) + : : : +- CometBroadcastHashJoin (88) + : : : :- CometFilter (86) + : : : : +- CometScan parquet spark_catalog.default.store_sales (85) + : : : +- ReusedExchange (87) + : : +- ReusedExchange (90) + : +- CometSort (96) + : +- ReusedExchange (95) + +- CometProject (112) + +- CometSortMergeJoin (111) + :- CometSort (108) + : +- CometExchange (107) + : +- CometProject (106) + : +- CometBroadcastHashJoin (105) + : :- CometProject (103) + : : +- CometBroadcastHashJoin (102) + : : :- CometFilter (100) + : : : +- CometScan parquet spark_catalog.default.web_sales (99) + : : +- ReusedExchange (101) + : +- ReusedExchange (104) + +- CometSort (110) + +- ReusedExchange (109) (1) CometScan parquet spark_catalog.default.catalog_sales @@ -401,7 +404,7 @@ Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_ 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: [] -(61) ColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [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] @@ -411,30 +414,33 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) Exchange +(63) 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] -(64) HashAggregate [codegen id : 2] +(64) CometColumnarToRow [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] + +(65) 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] -(65) Filter [codegen id : 2] +(66) 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) -(66) Exchange +(67) 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, [plan_id=9] +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] -(67) Sort [codegen id : 3] +(68) 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: [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 +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] -(68) CometScan parquet spark_catalog.default.catalog_sales +(69) CometScan 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 [] @@ -442,71 +448,71 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter +(70) 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) -(70) ReusedExchange [Reuses operator id: 6] +(71) 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] -(71) CometBroadcastHashJoin +(72) 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 -(72) CometProject +(73) 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] -(73) CometScan parquet spark_catalog.default.date_dim +(74) CometScan 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 -(74) CometFilter +(75) 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)) -(75) CometBroadcastExchange +(76) CometBroadcastExchange Input [2]: [d_date_sk#81, d_year#82] Arguments: [d_date_sk#81, d_year#82] -(76) CometBroadcastHashJoin +(77) 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 -(77) CometProject +(78) 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] -(78) CometExchange +(79) CometExchange 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, CometNativeShuffle, [plan_id=10] -(79) CometSort +(80) 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_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] -(80) ReusedExchange [Reuses operator id: 19] +(81) ReusedExchange [Reuses operator id: 19] Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] -(81) CometSort +(82) CometSort Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] 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] -(82) CometSortMergeJoin +(83) 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 -(83) CometProject +(84) 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] -(84) CometScan parquet spark_catalog.default.store_sales +(85) CometScan 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 [] @@ -514,59 +520,59 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter +(86) 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) -(86) ReusedExchange [Reuses operator id: 6] +(87) 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] -(87) CometBroadcastHashJoin +(88) 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 -(88) CometProject +(89) 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] -(89) ReusedExchange [Reuses operator id: 75] +(90) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#100, d_year#101] -(90) CometBroadcastHashJoin +(91) 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 -(91) CometProject +(92) 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] -(92) CometExchange +(93) CometExchange 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, CometNativeShuffle, [plan_id=11] -(93) CometSort +(94) 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: [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] -(94) ReusedExchange [Reuses operator id: 36] +(95) ReusedExchange [Reuses operator id: 36] Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] -(95) CometSort +(96) CometSort Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] 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] -(96) CometSortMergeJoin +(97) 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 -(97) CometProject +(98) 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] -(98) CometScan parquet spark_catalog.default.web_sales +(99) CometScan 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 [] @@ -574,148 +580,153 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_s PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter +(100) 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) -(100) ReusedExchange [Reuses operator id: 6] +(101) 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] -(101) CometBroadcastHashJoin +(102) 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 -(102) CometProject +(103) 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] -(103) ReusedExchange [Reuses operator id: 75] +(104) ReusedExchange [Reuses operator id: 76] Output [2]: [d_date_sk#119, d_year#120] -(104) CometBroadcastHashJoin +(105) 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 -(105) CometProject +(106) 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] -(106) CometExchange +(107) CometExchange 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, CometNativeShuffle, [plan_id=12] -(107) CometSort +(108) 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_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] -(108) ReusedExchange [Reuses operator id: 53] +(109) ReusedExchange [Reuses operator id: 53] Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] -(109) CometSort +(110) CometSort Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] 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] -(110) CometSortMergeJoin +(111) 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 -(111) CometProject +(112) 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] -(112) CometUnion +(113) 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] -(113) CometHashAggregate +(114) 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: [] -(114) CometExchange +(115) CometExchange 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, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate +(116) 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: [] -(116) ColumnarToRow [codegen id : 4] +(117) CometColumnarToRow [codegen id : 3] 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] -(117) HashAggregate [codegen id : 4] +(118) HashAggregate [codegen id : 3] 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] -(118) Exchange +(119) 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] + +(120) CometColumnarToRow [codegen id : 4] 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] -(119) HashAggregate [codegen id : 5] +(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, 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] -(120) Filter [codegen id : 5] +(122) Filter [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#131, sales_amt#132] Condition : isnotnull(sales_cnt#131) -(121) Exchange +(123) 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, [plan_id=15] +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] -(122) Sort [codegen id : 6] +(124) 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: [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 +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] -(123) 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) +(125) CometSortMergeJoin +Left output [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] +Right output [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#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80], Inner, ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) -(124) 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] +(126) CometProject 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] +Arguments: [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], [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] + +(127) CometTakeOrderedAndProject +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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#137 ASC NULLS FIRST,sales_amt_diff#138 ASC NULLS FIRST], output=[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]), [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], 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] -(125) TakeOrderedAndProject +(128) CometColumnarToRow [codegen id : 5] 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 (129) -+- * ColumnarToRow (128) - +- CometFilter (127) - +- CometScan parquet spark_catalog.default.date_dim (126) +BroadcastExchange (132) ++- * CometColumnarToRow (131) + +- CometFilter (130) + +- CometScan parquet spark_catalog.default.date_dim (129) -(126) CometScan parquet spark_catalog.default.date_dim +(129) CometScan 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 -(127) CometFilter +(130) 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)) -(128) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(129) BroadcastExchange +(132) BroadcastExchange Input [2]: [d_date_sk#13, d_year#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] @@ -723,33 +734,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 -BroadcastExchange (133) -+- * ColumnarToRow (132) - +- CometFilter (131) - +- CometScan parquet spark_catalog.default.date_dim (130) +Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (136) ++- * CometColumnarToRow (135) + +- CometFilter (134) + +- CometScan parquet spark_catalog.default.date_dim (133) -(130) CometScan parquet spark_catalog.default.date_dim +(133) CometScan 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 -(131) CometFilter +(134) 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)) -(132) ColumnarToRow [codegen id : 1] +(135) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#81, d_year#82] -(133) BroadcastExchange +(136) 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 = 84 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 +Subquery:5 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 +Subquery:6 Hosting operator id = 99 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 a80e7c7ef6..1c136ee8e3 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,20 +1,20 @@ -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 (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 (3) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (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] +WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] + CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,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 + 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] + CometColumnarToRow InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 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 + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 @@ -33,7 +33,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -85,19 +85,17 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i 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 (6) - Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - 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] + 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 + WholeStageCodegen (4) + 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] + CometColumnarToRow InputAdapter - Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - WholeStageCodegen (4) + CometColumnarExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 + WholeStageCodegen (3) HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 @@ -116,7 +114,7 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-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 5780ffc264..9d1259441d 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,115 @@ == 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) +* CometColumnarToRow (111) ++- CometTakeOrderedAndProject (110) + +- CometHashAggregate (109) + +- CometColumnarExchange (108) + +- * HashAggregate (107) + +- Union (106) + :- * HashAggregate (91) + : +- * CometColumnarToRow (90) + : +- CometColumnarExchange (89) + : +- * HashAggregate (88) + : +- Union (87) + : :- * Project (34) + : : +- * BroadcastHashJoin LeftOuter BuildRight (33) + : : :- * HashAggregate (18) + : : : +- * CometColumnarToRow (17) + : : : +- CometColumnarExchange (16) + : : : +- * HashAggregate (15) + : : : +- * CometColumnarToRow (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 (32) + : : +- * HashAggregate (31) + : : +- * CometColumnarToRow (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.store_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : :- * Project (55) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (54) + : : :- BroadcastExchange (44) + : : : +- * HashAggregate (43) + : : : +- * CometColumnarToRow (42) + : : : +- CometColumnarExchange (41) + : : : +- * HashAggregate (40) + : : : +- * CometColumnarToRow (39) + : : : +- CometProject (38) + : : : +- CometBroadcastHashJoin (37) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (35) + : : : +- ReusedExchange (36) + : : +- * HashAggregate (53) + : : +- * CometColumnarToRow (52) + : : +- CometColumnarExchange (51) + : : +- * HashAggregate (50) + : : +- * CometColumnarToRow (49) + : : +- CometProject (48) + : : +- CometBroadcastHashJoin (47) + : : :- CometScan parquet spark_catalog.default.catalog_returns (45) + : : +- ReusedExchange (46) + : +- * Project (86) + : +- * BroadcastHashJoin LeftOuter BuildRight (85) + : :- * HashAggregate (70) + : : +- * CometColumnarToRow (69) + : : +- CometColumnarExchange (68) + : : +- * HashAggregate (67) + : : +- * CometColumnarToRow (66) + : : +- CometProject (65) + : : +- CometBroadcastHashJoin (64) + : : :- CometProject (60) + : : : +- CometBroadcastHashJoin (59) + : : : :- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.web_sales (56) + : : : +- ReusedExchange (58) + : : +- CometBroadcastExchange (63) + : : +- CometFilter (62) + : : +- CometScan parquet spark_catalog.default.web_page (61) + : +- BroadcastExchange (84) + : +- * HashAggregate (83) + : +- * CometColumnarToRow (82) + : +- CometColumnarExchange (81) + : +- * HashAggregate (80) + : +- * CometColumnarToRow (79) + : +- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometProject (75) + : : +- CometBroadcastHashJoin (74) + : : :- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.web_returns (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + :- * HashAggregate (98) + : +- * CometColumnarToRow (97) + : +- CometColumnarExchange (96) + : +- * HashAggregate (95) + : +- * HashAggregate (94) + : +- * CometColumnarToRow (93) + : +- ReusedExchange (92) + +- * HashAggregate (105) + +- * CometColumnarToRow (104) + +- CometColumnarExchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- * CometColumnarToRow (100) + +- ReusedExchange (99) (1) CometScan parquet spark_catalog.default.store_sales @@ -164,7 +176,7 @@ Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] (15) HashAggregate [codegen id : 1] @@ -174,18 +186,21 @@ 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) CometColumnarExchange Input [3]: [s_store_sk#8, sum#11, sum#12] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 4] +(17) CometColumnarToRow [codegen id : 4] +Input [3]: [s_store_sk#8, sum#11, sum#12] + +(18) 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) CometScan parquet spark_catalog.default.store_returns +(19) CometScan 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 +208,171 @@ PartitionFilters: [isnotnull(sr_returned_date_sk#20), dynamicpruningexpression(s PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter +(20) 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] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#22] -(21) CometBroadcastHashJoin +(22) 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 +(23) 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] +(24) ReusedExchange [Reuses operator id: 11] Output [1]: [s_store_sk#23] -(24) CometBroadcastHashJoin +(25) 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 +(26) 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] +(27) CometColumnarToRow [codegen id : 2] Input [3]: [sr_return_amt#18, sr_net_loss#19, s_store_sk#23] -(27) HashAggregate [codegen id : 2] +(28) 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 +(29) CometColumnarExchange +Input [3]: [s_store_sk#23, sum#26, sum#27] +Arguments: hashpartitioning(s_store_sk#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] + +(30) CometColumnarToRow [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] +(31) 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 +(32) 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] +(33) 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] +(34) 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) CometScan parquet spark_catalog.default.catalog_sales +(35) CometScan 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] +(36) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#41] -(35) CometBroadcastHashJoin +(37) 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 +(38) 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] +(39) CometColumnarToRow [codegen id : 5] Input [3]: [cs_call_center_sk#36, cs_ext_sales_price#37, cs_net_profit#38] -(38) HashAggregate [codegen id : 5] +(40) 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 +(41) 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] +(42) CometColumnarToRow [codegen id : 6] +Input [3]: [cs_call_center_sk#36, sum#44, sum#45] + +(43) 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 +(44) BroadcastExchange Input [3]: [cs_call_center_sk#36, sales#48, profit#49] Arguments: IdentityBroadcastMode, [plan_id=5] -(42) CometScan parquet spark_catalog.default.catalog_returns +(45) CometScan 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] +(46) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#54] -(44) CometBroadcastHashJoin +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 7] Input [2]: [cr_return_amount#50, cr_net_loss#51] -(47) HashAggregate [codegen id : 7] +(50) 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 +(51) CometColumnarExchange Input [2]: [sum#57, sum#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(49) HashAggregate +(52) CometColumnarToRow +Input [2]: [sum#57, sum#58] + +(53) 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] +(54) BroadcastNestedLoopJoin [codegen id : 8] Join type: Inner Join condition: None -(51) Project [codegen id : 8] +(55) 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) CometScan parquet spark_catalog.default.web_sales +(56) CometScan 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 +380,71 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter +(57) 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] +(58) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#71] -(55) CometBroadcastHashJoin +(59) 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 +(60) 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) CometScan parquet spark_catalog.default.web_page +(61) CometScan 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 +(62) CometFilter Input [1]: [wp_web_page_sk#72] Condition : isnotnull(wp_web_page_sk#72) -(59) CometBroadcastExchange +(63) CometBroadcastExchange Input [1]: [wp_web_page_sk#72] Arguments: [wp_web_page_sk#72] -(60) CometBroadcastHashJoin +(64) 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 +(65) 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] +(66) CometColumnarToRow [codegen id : 9] Input [3]: [ws_ext_sales_price#67, ws_net_profit#68, wp_web_page_sk#72] -(63) HashAggregate [codegen id : 9] +(67) 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 +(68) CometColumnarExchange +Input [3]: [wp_web_page_sk#72, sum#75, sum#76] +Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(69) CometColumnarToRow [codegen id : 12] Input [3]: [wp_web_page_sk#72, sum#75, sum#76] -Arguments: hashpartitioning(wp_web_page_sk#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] -(65) HashAggregate [codegen id : 12] +(70) 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) CometScan parquet spark_catalog.default.web_returns +(71) CometScan 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 +452,228 @@ PartitionFilters: [isnotnull(wr_returned_date_sk#84), dynamicpruningexpression(w PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter +(72) 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] +(73) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#86] -(69) CometBroadcastHashJoin +(74) 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 +(75) 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] +(76) ReusedExchange [Reuses operator id: 63] Output [1]: [wp_web_page_sk#87] -(72) CometBroadcastHashJoin +(77) 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 +(78) 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] +(79) CometColumnarToRow [codegen id : 10] Input [3]: [wr_return_amt#82, wr_net_loss#83, wp_web_page_sk#87] -(75) HashAggregate [codegen id : 10] +(80) 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 +(81) CometColumnarExchange Input [3]: [wp_web_page_sk#87, sum#90, sum#91] -Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Arguments: hashpartitioning(wp_web_page_sk#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(77) HashAggregate [codegen id : 11] +(82) CometColumnarToRow [codegen id : 11] +Input [3]: [wp_web_page_sk#87, sum#90, sum#91] + +(83) 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 +(84) 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] +(85) 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] +(86) 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 +(87) Union -(82) HashAggregate [codegen id : 13] +(88) 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 +(89) 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, CometColumnarShuffle, [plan_id=10] + +(90) CometColumnarToRow [codegen id : 14] Input [8]: [channel#32, id#33, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#32, id#33, 5), ENSURE_REQUIREMENTS, [plan_id=10] -(84) HashAggregate [codegen id : 14] +(91) 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] +(92) ReusedExchange [Reuses operator id: 89] Output [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -(86) HashAggregate [codegen id : 28] +(93) CometColumnarToRow [codegen id : 28] +Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(94) 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] +(95) 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 +(96) 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] +(97) CometColumnarToRow [codegen id : 29] +Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(98) 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] +(99) ReusedExchange [Reuses operator id: 89] Output [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -(91) HashAggregate [codegen id : 43] +(100) CometColumnarToRow [codegen id : 43] +Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] + +(101) 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] +(102) 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 +(103) 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] +(104) CometColumnarToRow [codegen id : 44] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] + +(105) 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 +(106) Union -(96) HashAggregate [codegen id : 45] +(107) 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 +(108) 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, [plan_id=13] +Arguments: hashpartitioning(channel#32, id#33, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(98) HashAggregate [codegen id : 46] +(109) 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 +(110) 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] + +(111) CometColumnarToRow [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 (116) ++- * CometColumnarToRow (115) + +- CometProject (114) + +- CometFilter (113) + +- CometScan parquet spark_catalog.default.date_dim (112) -(100) CometScan parquet spark_catalog.default.date_dim +(112) CometScan 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 +(113) 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 +(114) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(103) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(104) BroadcastExchange +(116) 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 = 19 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 = 35 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 = 45 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 = 56 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 = 71 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 db4e102473..fdfca269ab 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,167 @@ -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] - 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 [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] +WholeStageCodegen (46) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow InputAdapter - Exchange [s_store_sk] #8 - WholeStageCodegen (2) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarExchange [s_store_sk] #3 + WholeStageCodegen (1) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + CometColumnarToRow 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] + 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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_sk] #8 + WholeStageCodegen (2) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cs_call_center_sk] #10 + WholeStageCodegen (5) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + CometColumnarToRow + 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 - 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] + ReusedExchange [d_date_sk] #5 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + CometColumnarToRow 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] - ColumnarToRow + CometColumnarExchange #11 + WholeStageCodegen (7) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow 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] + 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 - 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] - 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] - 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] + 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] + CometColumnarToRow InputAdapter - Exchange [wp_web_page_sk] #15 - WholeStageCodegen (10) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - ColumnarToRow + CometColumnarExchange [wp_web_page_sk] #12 + WholeStageCodegen (9) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + CometColumnarToRow 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] + 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 - 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [wp_web_page_sk] #15 + WholeStageCodegen (10) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #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] + CometColumnarToRow + 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 b15e683bd4..28a601b5a9 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,73 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- Exchange (20) - : : +- * HashAggregate (19) - : : +- * ColumnarToRow (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (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 (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- Exchange (40) - : +- * HashAggregate (39) - : +- * ColumnarToRow (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- Exchange (63) - +- * HashAggregate (62) - +- * ColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometProject (57) - : +- CometFilter (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometFilter (47) - : : +- CometScan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan parquet spark_catalog.default.catalog_returns (50) - +- ReusedExchange (58) +TakeOrderedAndProject (72) ++- * Project (71) + +- * SortMergeJoin Inner (70) + :- * Project (47) + : +- * SortMergeJoin Inner (46) + : :- * Sort (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometColumnarExchange (20) + : : +- * HashAggregate (19) + : : +- * CometColumnarToRow (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometSortMergeJoin (10) + : : : :- CometSort (4) + : : : : +- CometExchange (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometSort (9) + : : : +- CometExchange (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 (45) + : +- * Filter (44) + : +- * HashAggregate (43) + : +- * CometColumnarToRow (42) + : +- CometColumnarExchange (41) + : +- * HashAggregate (40) + : +- * CometColumnarToRow (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometFilter (34) + : : +- CometSortMergeJoin (33) + : : :- CometSort (27) + : : : +- CometExchange (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.web_sales (24) + : : +- CometSort (32) + : : +- CometExchange (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.web_returns (28) + : +- ReusedExchange (36) + +- * Sort (69) + +- * Filter (68) + +- * HashAggregate (67) + +- * CometColumnarToRow (66) + +- CometColumnarExchange (65) + +- * HashAggregate (64) + +- * CometColumnarToRow (63) + +- CometProject (62) + +- CometBroadcastHashJoin (61) + :- CometProject (59) + : +- CometFilter (58) + : +- CometSortMergeJoin (57) + : :- CometSort (51) + : : +- CometExchange (50) + : : +- CometFilter (49) + : : +- CometScan parquet spark_catalog.default.catalog_sales (48) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan parquet spark_catalog.default.catalog_returns (52) + +- ReusedExchange (60) (1) CometScan parquet spark_catalog.default.store_sales @@ -150,7 +153,7 @@ Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight 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] +(18) CometColumnarToRow [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] (19) HashAggregate [codegen id : 1] @@ -160,22 +163,25 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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] -(20) Exchange +(20) 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, [plan_id=3] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(21) HashAggregate [codegen id : 2] +(21) CometColumnarToRow [codegen id : 2] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(22) 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] -(22) Sort [codegen id : 2] +(23) 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 -(23) CometScan parquet spark_catalog.default.web_sales +(24) CometScan 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 [] @@ -183,106 +189,109 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter +(25) 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)) -(25) CometExchange +(26) CometExchange 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, CometNativeShuffle, [plan_id=4] -(26) CometSort +(27) 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_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) CometScan parquet spark_catalog.default.web_returns +(28) CometScan 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 +(29) 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 +(30) 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) CometExchange +(31) CometExchange Input [2]: [wr_item_sk#35, wr_order_number#36] Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(31) CometSort +(32) CometSort Input [2]: [wr_item_sk#35, wr_order_number#36] Arguments: [wr_item_sk#35, wr_order_number#36], [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST] -(32) CometSortMergeJoin +(33) 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 -(33) CometFilter +(34) 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) -(34) CometProject +(35) 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] -(35) ReusedExchange [Reuses operator id: 15] +(36) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#38, d_year#39] -(36) CometBroadcastHashJoin +(37) 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 -(37) CometProject +(38) 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] -(38) ColumnarToRow [codegen id : 3] +(39) CometColumnarToRow [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 : 3] +(40) 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 +(41) 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, CometColumnarShuffle, [plan_id=6] + +(42) CometColumnarToRow [codegen id : 4] 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 : 4] +(43) 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 : 4] +(44) 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 : 4] +(45) 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 : 5] +(46) 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 : 5] +(47) 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) CometScan parquet spark_catalog.default.catalog_sales +(48) CometScan 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 [] @@ -290,138 +299,141 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(47) CometFilter +(49) 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) CometExchange +(50) CometExchange 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, CometNativeShuffle, [plan_id=7] -(49) CometSort +(51) 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_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] -(50) CometScan parquet spark_catalog.default.catalog_returns +(52) CometScan 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 -(51) CometFilter +(53) 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)) -(52) CometProject +(54) 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] -(53) CometExchange +(55) CometExchange Input [2]: [cr_item_sk#62, cr_order_number#63] Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(54) CometSort +(56) CometSort Input [2]: [cr_item_sk#62, cr_order_number#63] Arguments: [cr_item_sk#62, cr_order_number#63], [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST] -(55) CometSortMergeJoin +(57) 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 -(56) CometFilter +(58) 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) -(57) CometProject +(59) 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] -(58) ReusedExchange [Reuses operator id: 15] +(60) ReusedExchange [Reuses operator id: 15] Output [2]: [d_date_sk#65, d_year#66] -(59) CometBroadcastHashJoin +(61) 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 -(60) CometProject +(62) 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] -(61) ColumnarToRow [codegen id : 6] +(63) CometColumnarToRow [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] -(62) HashAggregate [codegen id : 6] +(64) 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] -(63) Exchange +(65) 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] + +(66) CometColumnarToRow [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] -(64) HashAggregate [codegen id : 7] +(67) 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] -(65) Filter [codegen id : 7] +(68) 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) -(66) Sort [codegen id : 7] +(69) 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 -(67) SortMergeJoin [codegen id : 8] +(70) 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 -(68) Project [codegen id : 8] +(71) Project [codegen id : 8] Output [13]: [round((cast(ss_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(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] -(69) TakeOrderedAndProject +(72) 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 (73) -+- * ColumnarToRow (72) - +- CometFilter (71) - +- CometScan parquet spark_catalog.default.date_dim (70) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) -(70) CometScan parquet spark_catalog.default.date_dim +(73) CometScan 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 -(71) CometFilter +(74) 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)) -(72) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) BroadcastExchange +(76) 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 = 23 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 24 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 = 48 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 9e94433e61..6a47ffdd0e 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 @@ -10,87 +10,90 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp 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 (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] - CometExchange [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] - CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ss_item_sk,ss_customer_sk] #1 + 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] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 (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] #6 - 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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 (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] #9 - 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] - CometExchange [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] - CometExchange [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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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 07d7277d6a..6696198509 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,119 +1,128 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- Union (111) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * HashAggregate (38) - : : +- Exchange (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) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (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) - : : +- Exchange (66) - : : +- * HashAggregate (65) - : : +- * ColumnarToRow (64) - : : +- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometProject (60) - : : : +- CometBroadcastHashJoin (59) - : : : :- CometProject (57) - : : : : +- CometBroadcastHashJoin (56) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (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) - : +- Exchange (95) - : +- * HashAggregate (94) - : +- * ColumnarToRow (93) - : +- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (86) - : : : +- CometBroadcastHashJoin (85) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (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 (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - +- * HashAggregate (110) - +- Exchange (109) - +- * HashAggregate (108) - +- * HashAggregate (107) - +- ReusedExchange (106) +* CometColumnarToRow (124) ++- CometTakeOrderedAndProject (123) + +- CometHashAggregate (122) + +- CometColumnarExchange (121) + +- * HashAggregate (120) + +- Union (119) + :- * HashAggregate (104) + : +- * CometColumnarToRow (103) + : +- CometColumnarExchange (102) + : +- * HashAggregate (101) + : +- Union (100) + : :- * HashAggregate (39) + : : +- * CometColumnarToRow (38) + : : +- CometColumnarExchange (37) + : : +- * HashAggregate (36) + : : +- * CometColumnarToRow (35) + : : +- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometProject (28) + : : : +- CometBroadcastHashJoin (27) + : : : :- CometProject (22) + : : : : +- CometBroadcastHashJoin (21) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (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 (69) + : : +- * CometColumnarToRow (68) + : : +- CometColumnarExchange (67) + : : +- * HashAggregate (66) + : : +- * CometColumnarToRow (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (53) + : : : : : +- CometBroadcastHashJoin (52) + : : : : : :- CometProject (50) + : : : : : : +- CometSortMergeJoin (49) + : : : : : : :- CometSort (43) + : : : : : : : +- CometExchange (42) + : : : : : : : +- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- CometSort (48) + : : : : : : +- CometExchange (47) + : : : : : : +- CometProject (46) + : : : : : : +- CometFilter (45) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (44) + : : : : : +- ReusedExchange (51) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometFilter (55) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (54) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- * HashAggregate (99) + : +- * CometColumnarToRow (98) + : +- CometColumnarExchange (97) + : +- * HashAggregate (96) + : +- * CometColumnarToRow (95) + : +- CometProject (94) + : +- CometBroadcastHashJoin (93) + : :- CometProject (91) + : : +- CometBroadcastHashJoin (90) + : : :- CometProject (88) + : : : +- CometBroadcastHashJoin (87) + : : : :- CometProject (83) + : : : : +- CometBroadcastHashJoin (82) + : : : : :- CometProject (80) + : : : : : +- CometSortMergeJoin (79) + : : : : : :- CometSort (73) + : : : : : : +- CometExchange (72) + : : : : : : +- CometFilter (71) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (70) + : : : : : +- CometSort (78) + : : : : : +- CometExchange (77) + : : : : : +- CometProject (76) + : : : : : +- CometFilter (75) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (74) + : : : : +- ReusedExchange (81) + : : : +- CometBroadcastExchange (86) + : : : +- CometFilter (85) + : : : +- CometScan parquet spark_catalog.default.web_site (84) + : : +- ReusedExchange (89) + : +- ReusedExchange (92) + :- * HashAggregate (111) + : +- * CometColumnarToRow (110) + : +- CometColumnarExchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * CometColumnarToRow (106) + : +- ReusedExchange (105) + +- * HashAggregate (118) + +- * CometColumnarToRow (117) + +- CometColumnarExchange (116) + +- * HashAggregate (115) + +- * HashAggregate (114) + +- * CometColumnarToRow (113) + +- ReusedExchange (112) (1) CometScan parquet spark_catalog.default.store_sales @@ -276,7 +285,7 @@ Arguments: [ss_promo_sk#3], [p_promo_sk#20], Inner, BuildRight 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] -(35) ColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [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] (36) HashAggregate [codegen id : 1] @@ -286,18 +295,21 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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) Exchange +(37) 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, [plan_id=3] +Arguments: hashpartitioning(s_store_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(38) HashAggregate [codegen id : 2] +(38) CometColumnarToRow [codegen id : 2] +Input [6]: [s_store_id#17, sum#27, sum#28, isEmpty#29, sum#30, isEmpty#31] + +(39) 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))#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] -(39) CometScan parquet spark_catalog.default.catalog_sales +(40) CometScan 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 [] @@ -305,132 +317,135 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(40) CometFilter +(41) 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)) -(41) CometExchange +(42) CometExchange 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, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) 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] -(43) CometScan parquet spark_catalog.default.catalog_returns +(44) CometScan parquet spark_catalog.default.catalog_returns 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 -(44) CometFilter +(45) CometFilter 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)) -(45) CometProject +(46) CometProject 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] -(46) CometExchange +(47) CometExchange 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, CometNativeShuffle, [plan_id=5] -(47) CometSort +(48) 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] -(48) CometSortMergeJoin +(49) 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 -(49) CometProject +(50) 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] -(50) ReusedExchange [Reuses operator id: 15] +(51) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#53] -(51) CometBroadcastHashJoin +(52) 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 -(52) CometProject +(53) 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] -(53) CometScan parquet spark_catalog.default.catalog_page +(54) CometScan 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 -(54) CometFilter +(55) CometFilter Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Condition : isnotnull(cp_catalog_page_sk#54) -(55) CometBroadcastExchange +(56) CometBroadcastExchange Input [2]: [cp_catalog_page_sk#54, cp_catalog_page_id#55] Arguments: [cp_catalog_page_sk#54, cp_catalog_page_id#55] -(56) CometBroadcastHashJoin +(57) 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 -(57) CometProject +(58) 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] -(58) ReusedExchange [Reuses operator id: 26] +(59) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#56] -(59) CometBroadcastHashJoin +(60) 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 -(60) CometProject +(61) 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] -(61) ReusedExchange [Reuses operator id: 32] +(62) ReusedExchange [Reuses operator id: 32] Output [1]: [p_promo_sk#57] -(62) CometBroadcastHashJoin +(63) 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 -(63) CometProject +(64) 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] -(64) ColumnarToRow [codegen id : 3] +(65) CometColumnarToRow [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] -(65) HashAggregate [codegen id : 3] +(66) 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] -(66) Exchange +(67) 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, [plan_id=6] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(67) HashAggregate [codegen id : 4] +(68) CometColumnarToRow [codegen id : 4] +Input [6]: [cp_catalog_page_id#55, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] + +(69) 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] -(68) CometScan parquet spark_catalog.default.web_sales +(70) CometScan 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 [] @@ -438,265 +453,284 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#82), dynamicpruningexpression(ws_so PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(69) CometFilter +(71) 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)) -(70) CometExchange +(72) CometExchange 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, CometNativeShuffle, [plan_id=7] -(71) CometSort +(73) 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] -(72) CometScan parquet spark_catalog.default.web_returns +(74) CometScan parquet spark_catalog.default.web_returns 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 -(73) CometFilter +(75) CometFilter 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)) -(74) CometProject +(76) CometProject 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] -(75) CometExchange +(77) CometExchange 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, CometNativeShuffle, [plan_id=8] -(76) CometSort +(78) 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] -(77) CometSortMergeJoin +(79) 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 -(78) CometProject +(80) 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] -(79) ReusedExchange [Reuses operator id: 15] +(81) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#89] -(80) CometBroadcastHashJoin +(82) 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 -(81) CometProject +(83) 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] -(82) CometScan parquet spark_catalog.default.web_site +(84) CometScan 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 -(83) CometFilter +(85) CometFilter Input [2]: [web_site_sk#90, web_site_id#91] Condition : isnotnull(web_site_sk#90) -(84) CometBroadcastExchange +(86) CometBroadcastExchange Input [2]: [web_site_sk#90, web_site_id#91] Arguments: [web_site_sk#90, web_site_id#91] -(85) CometBroadcastHashJoin +(87) 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 -(86) CometProject +(88) 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] -(87) ReusedExchange [Reuses operator id: 26] +(89) ReusedExchange [Reuses operator id: 26] Output [1]: [i_item_sk#92] -(88) CometBroadcastHashJoin +(90) 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 -(89) CometProject +(91) 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] -(90) ReusedExchange [Reuses operator id: 32] +(92) ReusedExchange [Reuses operator id: 32] Output [1]: [p_promo_sk#93] -(91) CometBroadcastHashJoin +(93) 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 -(92) CometProject +(94) 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] -(93) ColumnarToRow [codegen id : 5] +(95) CometColumnarToRow [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] -(94) HashAggregate [codegen id : 5] +(96) 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] -(95) Exchange +(97) 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] + +(98) CometColumnarToRow [codegen id : 6] 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, [plan_id=9] -(96) HashAggregate [codegen id : 6] +(99) 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] -(97) Union +(100) Union -(98) HashAggregate [codegen id : 7] +(101) 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] -(99) Exchange +(102) 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] + +(103) CometColumnarToRow [codegen id : 8] 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, [plan_id=10] -(100) HashAggregate [codegen id : 8] +(104) 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] -(101) ReusedExchange [Reuses operator id: 99] +(105) ReusedExchange [Reuses operator id: 102] Output [8]: [channel#130, id#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -(102) HashAggregate [codegen id : 16] +(106) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#130, id#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] + +(107) 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] -(103) HashAggregate [codegen id : 16] +(108) 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] -(104) Exchange +(109) CometColumnarExchange Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Arguments: hashpartitioning(channel#130, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(channel#130, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(105) HashAggregate [codegen id : 17] +(110) CometColumnarToRow [codegen id : 17] +Input [7]: [channel#130, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] + +(111) 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] -(106) ReusedExchange [Reuses operator id: 99] +(112) ReusedExchange [Reuses operator id: 102] Output [8]: [channel#163, id#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -(107) HashAggregate [codegen id : 25] +(113) CometColumnarToRow [codegen id : 25] +Input [8]: [channel#163, id#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] + +(114) 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] -(108) HashAggregate [codegen id : 25] +(115) HashAggregate [codegen id : 25] Input [3]: [sales#174, returns#175, profit#176] Keys: [] 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] -(109) Exchange +(116) CometColumnarExchange +Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(117) CometColumnarToRow [codegen id : 26] Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(110) HashAggregate [codegen id : 26] +(118) HashAggregate [codegen id : 26] Input [6]: [sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] Keys: [] 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] -(111) Union +(119) Union -(112) HashAggregate [codegen id : 27] +(120) 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#35, id#36, sales#127, returns#128, profit#129] -(113) Exchange +(121) 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, [plan_id=13] +Arguments: hashpartitioning(channel#35, id#36, sales#127, returns#128, profit#129, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(114) HashAggregate [codegen id : 28] +(122) 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#35, id#36, sales#127, returns#128, profit#129] -(115) TakeOrderedAndProject +(123) 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] + +(124) CometColumnarToRow [codegen id : 28] Input [5]: [channel#35, id#36, sales#127, returns#128, profit#129] -Arguments: 100, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [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 (120) -+- * ColumnarToRow (119) - +- CometProject (118) - +- CometFilter (117) - +- CometScan parquet spark_catalog.default.date_dim (116) +BroadcastExchange (129) ++- * CometColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) -(116) CometScan parquet spark_catalog.default.date_dim +(125) CometScan 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 -(117) CometFilter +(126) 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)) -(118) CometProject +(127) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(128) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(120) BroadcastExchange +(129) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#82 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 70 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 75dbc06d7f..5cd4255061 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,153 +1,162 @@ -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,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) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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,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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_store_id] #3 + 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] + CometColumnarToRow + 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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [cp_catalog_page_id] #11 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [web_site_id] #15 + 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] + CometColumnarToRow + 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] + CometExchange [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] + CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + 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] + CometColumnarToRow InputAdapter - Exchange [s_store_id] #3 - 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] - CometExchange [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] - CometExchange [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] + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #20 + 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] + CometColumnarToRow InputAdapter - Exchange [cp_catalog_page_id] #11 - 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] - CometExchange [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] - CometExchange [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] - InputAdapter - Exchange [web_site_id] #15 - 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] - CometExchange [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] - CometExchange [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] - InputAdapter - Exchange [channel] #19 - 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,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Exchange #20 - 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 + 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 3f5b2d6b2b..c5e30830b1 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,46 @@ == 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 (42) ++- * Project (41) + +- Window (40) + +- * CometColumnarToRow (39) + +- CometSort (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometColumnarExchange (35) + +- * HashAggregate (34) + +- Union (33) + :- * HashAggregate (18) + : +- * CometColumnarToRow (17) + : +- CometColumnarExchange (16) + : +- * HashAggregate (15) + : +- * CometColumnarToRow (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 (25) + : +- * CometColumnarToRow (24) + : +- CometColumnarExchange (23) + : +- * HashAggregate (22) + : +- * HashAggregate (21) + : +- * CometColumnarToRow (20) + : +- ReusedExchange (19) + +- * HashAggregate (32) + +- * CometColumnarToRow (31) + +- CometColumnarExchange (30) + +- * HashAggregate (29) + +- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- ReusedExchange (26) (1) CometScan parquet spark_catalog.default.web_sales @@ -101,7 +107,7 @@ Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#8, i_category#9] Arguments: [ws_net_paid#2, i_class#8, i_category#9], [ws_net_paid#2, i_class#8, i_category#9] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [3]: [ws_net_paid#2, i_class#8, i_category#9] (15) HashAggregate [codegen id : 1] @@ -111,142 +117,158 @@ 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) CometColumnarExchange Input [3]: [i_category#9, i_class#8, sum#11] -Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, [plan_id=1] +Arguments: hashpartitioning(i_category#9, i_class#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [codegen id : 2] +Input [3]: [i_category#9, i_class#8, sum#11] + +(18) 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] +(19) ReusedExchange [Reuses operator id: 16] Output [3]: [i_category#17, i_class#18, sum#19] -(19) HashAggregate [codegen id : 4] +(20) CometColumnarToRow [codegen id : 4] +Input [3]: [i_category#17, i_class#18, sum#19] + +(21) 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] +(22) 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 +(23) CometColumnarExchange Input [3]: [i_category#17, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] +Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] -(22) HashAggregate [codegen id : 5] +(24) CometColumnarToRow [codegen id : 5] +Input [3]: [i_category#17, sum#24, isEmpty#25] + +(25) 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] +(26) ReusedExchange [Reuses operator id: 16] Output [3]: [i_category#32, i_class#33, sum#34] -(24) HashAggregate [codegen id : 7] +(27) CometColumnarToRow [codegen id : 7] +Input [3]: [i_category#32, i_class#33, sum#34] + +(28) 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] +(29) 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 +(30) CometColumnarExchange +Input [2]: [sum#39, isEmpty#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] + +(31) CometColumnarToRow [codegen id : 8] Input [2]: [sum#39, isEmpty#40] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] -(27) HashAggregate [codegen id : 8] +(32) 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 +(33) Union -(29) HashAggregate [codegen id : 9] +(34) 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 +(35) 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, [plan_id=4] +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] -(31) HashAggregate [codegen id : 10] +(36) 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 +(37) CometExchange +Input [5]: [total_sum#13, i_category#9, i_class#8, lochierarchy#16, _w0#48] +Arguments: hashpartitioning(lochierarchy#16, _w0#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(38) 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] +(39) CometColumnarToRow [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 +(40) 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] +(41) 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 +(42) 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 (47) ++- * CometColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) -(37) CometScan parquet spark_catalog.default.date_dim +(43) CometScan 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 +(44) 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 +(45) 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] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(41) BroadcastExchange +(47) 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 8996119f2c..4ea8a6deef 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,27 +1,27 @@ 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) + CometColumnarToRow 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] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [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 + 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] + CometColumnarToRow InputAdapter - Exchange [i_category,i_class] #3 + CometColumnarExchange [i_category,i_class] #3 WholeStageCodegen (1) HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [ws_net_paid,i_class,i_category] CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -44,21 +44,25 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl 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] + WholeStageCodegen (5) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow InputAdapter - Exchange [i_category] #7 + CometColumnarExchange [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] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #8 + CometColumnarExchange #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 + CometColumnarToRow + 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 61ef88e69c..4977b112bc 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,28 +1,30 @@ == Physical Plan == -* ColumnarToRow (24) -+- CometSort (23) - +- CometColumnarExchange (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) +* CometColumnarToRow (26) ++- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometColumnarExchange (16) + +- * HashAggregate (15) + +- * CometColumnarToRow (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) CometScan parquet spark_catalog.default.store_sales @@ -89,7 +91,7 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] Arguments: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10], [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(14) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] (15) HashAggregate [codegen id : 1] @@ -99,73 +101,79 @@ 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) 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, [plan_id=1] +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] -(17) HashAggregate [codegen id : 2] +(17) CometColumnarToRow [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] + +(18) 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 +(19) 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] + +(20) 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] +(21) CometColumnarToRow [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 +(22) 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] +(23) 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) CometColumnarExchange +(24) 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] -(23) CometSort +(25) 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: [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] -(24) ColumnarToRow [codegen id : 5] +(26) CometColumnarToRow [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] ===== 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) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.date_dim (27) -(25) CometScan parquet spark_catalog.default.date_dim +(27) CometScan 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) CometColumnarToRow [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-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 a08ff0a3a2..e3a2179096 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -8,35 +8,37 @@ WholeStageCodegen (5) InputAdapter Window [_w0,i_class] WholeStageCodegen (3) - Sort [i_class] + CometColumnarToRow 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 [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,_w0] + CometColumnarExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 868b2f4822..c2b7f49205 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (39) +* CometColumnarToRow (39) +- CometTakeOrderedAndProject (38) +- CometHashAggregate (37) +- CometExchange (36) @@ -220,14 +220,14 @@ Functions [1]: [count(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] 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] +(39) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (44) -+- * ColumnarToRow (43) ++- * CometColumnarToRow (43) +- CometProject (42) +- CometFilter (41) +- CometScan parquet spark_catalog.default.date_dim (40) @@ -248,7 +248,7 @@ Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 200 Input [3]: [d_date_sk#7, d_year#8, d_moy#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(43) ColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (44) BroadcastExchange 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 2e8f2022cf..b570b700ac 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] 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 689697306d..0bcd9798e8 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#35, customer_first_name#36, customer_last_name#37, custo 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#14, d_year#15] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#31 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (76) BroadcastExchange 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 8743dd42ef..987c965c03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index a428886298..b8d68b0a31 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 6f51be0c5b..4a829e8612 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 31a11bba90..5c694fb53c 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (85) +* CometColumnarToRow (85) +- CometTakeOrderedAndProject (84) +- CometBroadcastHashJoin (83) :- CometFilter (64) @@ -480,13 +480,13 @@ Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_c 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] +(85) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* ColumnarToRow (102) +* CometColumnarToRow (102) +- CometHashAggregate (101) +- CometExchange (100) +- CometHashAggregate (99) @@ -581,7 +581,7 @@ Input [2]: [sum#95, count#96] Keys: [] Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -(102) ColumnarToRow [codegen id : 1] +(102) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#97] Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 @@ -592,7 +592,7 @@ Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN d Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (107) -+- * ColumnarToRow (106) ++- * CometColumnarToRow (106) +- CometProject (105) +- CometFilter (104) +- CometScan parquet spark_catalog.default.date_dim (103) @@ -613,7 +613,7 @@ Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subq Input [2]: [d_date_sk#43, d_week_seq#44] Arguments: [d_date_sk#43], [d_date_sk#43] -(106) ColumnarToRow [codegen id : 1] +(106) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (107) BroadcastExchange @@ -621,7 +621,7 @@ Input [1]: [d_date_sk#43] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* ColumnarToRow (111) +* CometColumnarToRow (111) +- CometProject (110) +- CometFilter (109) +- CometScan parquet spark_catalog.default.date_dim (108) @@ -642,12 +642,12 @@ Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_ Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] Arguments: [d_week_seq#98], [d_week_seq#98] -(111) ColumnarToRow [codegen id : 1] +(111) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#98] Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (116) -+- * ColumnarToRow (115) ++- * CometColumnarToRow (115) +- CometProject (114) +- CometFilter (113) +- CometScan parquet spark_catalog.default.date_dim (112) @@ -668,7 +668,7 @@ Condition : (((isnotnull(d_year#102) AND (d_year#102 >= 1998)) AND (d_year#102 < Input [2]: [d_date_sk#26, d_year#102] Arguments: [d_date_sk#26], [d_date_sk#26] -(115) ColumnarToRow [codegen id : 1] +(115) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (116) BroadcastExchange @@ -685,7 +685,7 @@ Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquer Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 BroadcastExchange (121) -+- * ColumnarToRow (120) ++- * CometColumnarToRow (120) +- CometProject (119) +- CometFilter (118) +- CometScan parquet spark_catalog.default.date_dim (117) @@ -706,7 +706,7 @@ Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subq Input [2]: [d_date_sk#64, d_week_seq#65] Arguments: [d_date_sk#64], [d_date_sk#64] -(120) ColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#64] (121) BroadcastExchange @@ -714,7 +714,7 @@ Input [1]: [d_date_sk#64] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* ColumnarToRow (125) +* CometColumnarToRow (125) +- CometProject (124) +- CometFilter (123) +- CometScan parquet spark_catalog.default.date_dim (122) @@ -735,7 +735,7 @@ Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] Arguments: [d_week_seq#103], [d_week_seq#103] -(125) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_week_seq#103] 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 bb4be4d3ae..befc6dadbb 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,12 +1,12 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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 + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 @@ -40,13 +40,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] @@ -71,7 +71,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #3 BroadcastExchange #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -132,13 +132,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #5 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_week_seq] Subquery #6 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_week_seq] CometFilter [d_week_seq,d_year,d_moy,d_dom] 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 4bc928e635..124efb8084 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (122) +* CometColumnarToRow (122) +- CometTakeOrderedAndProject (121) +- CometHashAggregate (120) +- CometExchange (119) @@ -682,13 +682,13 @@ Functions: [] 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] +(122) CometColumnarToRow [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 = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* ColumnarToRow (142) +* CometColumnarToRow (142) +- CometHashAggregate (141) +- CometExchange (140) +- CometHashAggregate (139) @@ -802,14 +802,14 @@ Input [2]: [sum#143, count#144] Keys: [] Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] -(142) ColumnarToRow [codegen id : 1] +(142) CometColumnarToRow [codegen id : 1] Input [1]: [average_sales#145] Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 BroadcastExchange (147) -+- * ColumnarToRow (146) ++- * CometColumnarToRow (146) +- CometProject (145) +- CometFilter (144) +- CometScan parquet spark_catalog.default.date_dim (143) @@ -830,7 +830,7 @@ Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 < Input [2]: [d_date_sk#132, d_year#133] Arguments: [d_date_sk#132], [d_date_sk#132] -(146) ColumnarToRow [codegen id : 1] +(146) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#132] (147) BroadcastExchange @@ -841,7 +841,7 @@ Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 BroadcastExchange (152) -+- * ColumnarToRow (151) ++- * CometColumnarToRow (151) +- CometProject (150) +- CometFilter (149) +- CometScan parquet spark_catalog.default.date_dim (148) @@ -862,7 +862,7 @@ Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2 Input [3]: [d_date_sk#43, d_year#44, d_moy#45] Arguments: [d_date_sk#43], [d_date_sk#43] -(151) ColumnarToRow [codegen id : 1] +(151) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#43] (152) BroadcastExchange @@ -871,7 +871,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)) Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 BroadcastExchange (157) -+- * ColumnarToRow (156) ++- * CometColumnarToRow (156) +- CometProject (155) +- CometFilter (154) +- CometScan parquet spark_catalog.default.date_dim (153) @@ -892,7 +892,7 @@ Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 < Input [2]: [d_date_sk#26, d_year#146] Arguments: [d_date_sk#26], [d_date_sk#26] -(156) ColumnarToRow [codegen id : 1] +(156) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#26] (157) BroadcastExchange 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 760d0963ed..162f4f2447 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 @@ -30,7 +30,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #4 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] @@ -57,7 +57,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] @@ -81,7 +81,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 5c25745ba0..6275dfe15e 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (143) +* CometColumnarToRow (143) +- CometTakeOrderedAndProject (142) +- CometUnion (141) :- CometHashAggregate (38) @@ -785,14 +785,14 @@ Child 4 Input [11]: [i_item_id#144, ca_country#145, ca_state#146, county#147, ag 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] -(143) ColumnarToRow [codegen id : 1] +(143) CometColumnarToRow [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 (148) -+- * ColumnarToRow (147) ++- * CometColumnarToRow (147) +- CometProject (146) +- CometFilter (145) +- CometScan parquet spark_catalog.default.date_dim (144) @@ -813,7 +813,7 @@ Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_ Input [2]: [d_date_sk#25, d_year#26] Arguments: [d_date_sk#25], [d_date_sk#25] -(147) ColumnarToRow [codegen id : 1] +(147) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#25] (148) BroadcastExchange 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 add76afad3..b974a7afce 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 a0c02e00e9..0610ff5ce3 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,7 +2,7 @@ TakeOrderedAndProject (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -109,7 +109,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -128,7 +128,7 @@ Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_ite Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (27) -+- * ColumnarToRow (26) ++- * CometColumnarToRow (26) +- CometProject (25) +- CometFilter (24) +- CometScan parquet spark_catalog.default.date_dim (23) @@ -149,7 +149,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(26) ColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (27) BroadcastExchange 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 c4d5e48ae5..8e639b6fa0 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 @@ -20,7 +20,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [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 34d510fc71..1afbec79bb 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,29 @@ == 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 (25) ++- * HashAggregate (24) + +- * CometColumnarToRow (23) + +- CometColumnarExchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastNestedLoopJoin Inner BuildRight (18) + :- * CometColumnarToRow (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) + +- * CometColumnarToRow (16) + +- CometScan parquet spark_catalog.default.warehouse (15) (1) CometScan parquet spark_catalog.default.inventory @@ -89,7 +90,7 @@ Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] Arguments: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11], [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] -(14) ColumnarToRow [codegen id : 2] +(14) CometColumnarToRow [codegen id : 2] Input [5]: [inv_quantity_on_hand#2, i_brand#8, i_class#9, i_category#10, i_product_name#11] (15) CometScan parquet spark_catalog.default.warehouse @@ -98,7 +99,7 @@ Batched: true Location [not included in comparison]/{warehouse_dir}/warehouse] ReadSchema: struct<> -(16) ColumnarToRow [codegen id : 1] +(16) CometColumnarToRow [codegen id : 1] Input: [] (17) BroadcastExchange @@ -124,50 +125,53 @@ 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) 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, [plan_id=2] +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] -(23) HashAggregate [codegen id : 3] +(23) CometColumnarToRow [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] + +(24) 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 +(25) 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 (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) -(25) CometScan parquet spark_catalog.default.date_dim +(26) CometScan 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 +(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)) -(27) CometProject +(28) 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] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(29) BroadcastExchange +(30) 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 0e864ab701..7fed747fe6 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,40 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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 + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 b284c9b275..ce03b5946a 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (46) +* CometColumnarToRow (46) +- CometTakeOrderedAndProject (45) +- CometUnion (44) :- CometHashAggregate (23) @@ -258,14 +258,14 @@ Child 4 Input [5]: [i_product_name#37, i_brand#38, i_class#39, i_category#40, qo 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) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometProject (49) +- CometFilter (48) +- CometScan parquet spark_catalog.default.date_dim (47) @@ -286,7 +286,7 @@ Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_mont Input [2]: [d_date_sk#6, d_month_seq#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] (51) BroadcastExchange 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 63eda27a6b..02f615b128 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 15584fda11..8a25d9445e 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,49 +1,51 @@ == Physical Plan == -* ColumnarToRow (45) -+- CometSort (44) - +- CometColumnarExchange (43) - +- * Filter (42) - +- * HashAggregate (41) - +- Exchange (40) - +- * HashAggregate (39) - +- * HashAggregate (38) - +- Exchange (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) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (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) +* CometColumnarToRow (47) ++- CometSort (46) + +- CometColumnarExchange (45) + +- * Filter (44) + +- * HashAggregate (43) + +- * CometColumnarToRow (42) + +- CometColumnarExchange (41) + +- * HashAggregate (40) + +- * HashAggregate (39) + +- * CometColumnarToRow (38) + +- CometColumnarExchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * CometColumnarToRow (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (18) + : : : +- CometBroadcastHashJoin (17) + : : : :- CometProject (12) + : : : : +- CometSortMergeJoin (11) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometSort (10) + : : : : +- CometExchange (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) + +- * CometColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.customer_address (30) (1) CometScan parquet spark_catalog.default.store_sales @@ -177,7 +179,7 @@ Arguments: [ss_customer_sk#2], [c_customer_sk#21], Inner, BuildRight 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] +(29) CometColumnarToRow [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] (30) CometScan parquet spark_catalog.default.customer_address @@ -191,7 +193,7 @@ ReadSchema: struct cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) -(43) CometColumnarExchange +(45) 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, CometColumnarShuffle, [plan_id=6] -(44) CometSort +(46) 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] -(45) ColumnarToRow [codegen id : 5] +(47) CometColumnarToRow [codegen id : 5] Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] ===== Subqueries ===== -Subquery:1 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (72) -+- Exchange (71) - +- * HashAggregate (70) - +- * HashAggregate (69) - +- Exchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * ColumnarToRow (63) - : +- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (59) - : : +- CometBroadcastHashJoin (58) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometProject (51) - : : : : +- CometSortMergeJoin (50) - : : : : :- CometSort (47) - : : : : : +- ReusedExchange (46) - : : : : +- CometSort (49) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (52) - : : +- CometBroadcastExchange (57) - : : +- CometFilter (56) - : : +- CometScan parquet spark_catalog.default.item (55) - : +- ReusedExchange (60) - +- ReusedExchange (64) - - -(46) ReusedExchange [Reuses operator id: 4] +Subquery:1 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (76) ++- * CometColumnarToRow (75) + +- CometColumnarExchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- * CometColumnarToRow (71) + +- CometColumnarExchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * CometColumnarToRow (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (56) + : : : +- CometBroadcastHashJoin (55) + : : : :- CometProject (53) + : : : : +- CometSortMergeJoin (52) + : : : : :- CometSort (49) + : : : : : +- ReusedExchange (48) + : : : : +- CometSort (51) + : : : : +- ReusedExchange (50) + : : : +- ReusedExchange (54) + : : +- CometBroadcastExchange (59) + : : +- CometFilter (58) + : : +- CometScan parquet spark_catalog.default.item (57) + : +- ReusedExchange (62) + +- ReusedExchange (66) + + +(48) 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] -(47) CometSort +(49) 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_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] -(48) ReusedExchange [Reuses operator id: 9] +(50) ReusedExchange [Reuses operator id: 9] Output [2]: [sr_item_sk#7, sr_ticket_number#8] -(49) CometSort +(51) CometSort Input [2]: [sr_item_sk#7, sr_ticket_number#8] Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] -(50) CometSortMergeJoin +(52) 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 -(51) CometProject +(53) 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] -(52) ReusedExchange [Reuses operator id: 16] +(54) ReusedExchange [Reuses operator id: 16] Output [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] -(53) CometBroadcastHashJoin +(55) 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 -(54) CometProject +(56) 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] -(55) CometScan parquet spark_catalog.default.item +(57) CometScan 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 -(56) CometFilter +(58) 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) -(57) CometBroadcastExchange +(59) 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] -(58) CometBroadcastHashJoin +(60) 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 -(59) CometProject +(61) 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] -(60) ReusedExchange [Reuses operator id: 26] +(62) 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] -(61) CometBroadcastHashJoin +(63) 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 -(62) CometProject +(64) 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] -(63) ColumnarToRow [codegen id : 2] +(65) CometColumnarToRow [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] -(64) ReusedExchange [Reuses operator id: 33] +(66) ReusedExchange [Reuses operator id: 33] Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] -(65) BroadcastHashJoin [codegen id : 2] +(67) 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 -(66) Project [codegen id : 2] +(68) 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] -(67) HashAggregate [codegen id : 2] +(69) 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] -(68) Exchange +(70) 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, CometColumnarShuffle, [plan_id=7] + +(71) CometColumnarToRow [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] -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] -(69) HashAggregate [codegen id : 3] +(72) 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] -(70) HashAggregate [codegen id : 3] +(73) 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] -(71) Exchange +(74) CometColumnarExchange +Input [2]: [sum#46, count#47] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] + +(75) CometColumnarToRow [codegen id : 4] Input [2]: [sum#46, count#47] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] -(72) HashAggregate [codegen id : 4] +(76) 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 f4bcae0f13..27e6a9cd77 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,5 +1,5 @@ WholeStageCodegen (5) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,s_store_name,paid] CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 @@ -8,18 +8,55 @@ WholeStageCodegen (5) Subquery #1 WholeStageCodegen (4) HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - InputAdapter - Exchange #10 - 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #10 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + 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 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 + 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] + CometColumnarToRow 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 + 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 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 + CometColumnarToRow 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] @@ -30,62 +67,29 @@ WholeStageCodegen (5) 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 + CometExchange [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] - 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 + CometExchange [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] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + 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 - 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 (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] - 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 (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] - CometExchange [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] - CometExchange [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 #9 - WholeStageCodegen (1) - 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] + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + 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] 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 2327148b4b..85ad5c24bf 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (69) +* CometColumnarToRow (69) +- CometTakeOrderedAndProject (68) +- CometUnion (67) :- CometHashAggregate (27) @@ -385,14 +385,14 @@ Child 2 Input [7]: [i_item_id#61, s_state#62, g_state#63, agg1#64, agg2#65, agg3 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] -(69) ColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [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 (74) -+- * ColumnarToRow (73) ++- * CometColumnarToRow (73) +- CometProject (72) +- CometFilter (71) +- CometScan parquet spark_catalog.default.date_dim (70) @@ -413,7 +413,7 @@ Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_ Input [2]: [d_date_sk#14, d_year#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(73) ColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] (74) BroadcastExchange 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 ed169acda8..8adbfa9ee8 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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] @@ -19,7 +19,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 2d2b384c87..25e9930cce 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (32) +* CometColumnarToRow (32) +- CometSort (31) +- CometColumnarExchange (30) +- CometProject (29) @@ -179,14 +179,14 @@ Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC 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] -(32) ColumnarToRow [codegen id : 1] +(32) CometColumnarToRow [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 (37) -+- * ColumnarToRow (36) ++- * CometColumnarToRow (36) +- CometProject (35) +- CometFilter (34) +- CometScan parquet spark_catalog.default.date_dim (33) @@ -207,7 +207,7 @@ Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_do Input [3]: [d_date_sk#7, d_year#8, d_dom#9] Arguments: [d_date_sk#7], [d_date_sk#7] -(36) ColumnarToRow [codegen id : 1] +(36) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (37) BroadcastExchange 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 7da6076f5b..e1fc6bf43f 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -20,7 +20,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_dom] 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 5933ea5fad..cc7702fe7b 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,49 @@ == 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 (45) ++- * HashAggregate (44) + +- * CometColumnarToRow (43) + +- CometColumnarExchange (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) + : : : :- * CometColumnarToRow (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) + : : : +- * CometColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometBroadcastHashJoin (15) + : : : :- CometScan parquet spark_catalog.default.web_sales (13) + : : : +- ReusedExchange (14) + : : +- BroadcastExchange (25) + : : +- * CometColumnarToRow (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometScan parquet spark_catalog.default.catalog_sales (20) + : : +- ReusedExchange (21) + : +- BroadcastExchange (32) + : +- * CometColumnarToRow (31) + : +- CometFilter (30) + : +- CometScan parquet spark_catalog.default.customer_address (29) + +- BroadcastExchange (38) + +- * CometColumnarToRow (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.customer_demographics (35) (1) CometScan parquet spark_catalog.default.customer @@ -100,7 +101,7 @@ Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] Right output [1]: [ss_customer_sk#6] Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight -(12) ColumnarToRow [codegen id : 5] +(12) CometColumnarToRow [codegen id : 5] Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] (13) CometScan parquet spark_catalog.default.web_sales @@ -122,7 +123,7 @@ Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] -(17) ColumnarToRow [codegen id : 1] +(17) CometColumnarToRow [codegen id : 1] Input [1]: [ws_bill_customer_sk#12] (18) BroadcastExchange @@ -154,7 +155,7 @@ Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] -(24) ColumnarToRow [codegen id : 2] +(24) CometColumnarToRow [codegen id : 2] Input [1]: [cs_ship_customer_sk#16] (25) BroadcastExchange @@ -186,7 +187,7 @@ ReadSchema: struct Input [2]: [ca_address_sk#20, ca_state#21] Condition : isnotnull(ca_address_sk#20) -(31) ColumnarToRow [codegen id : 3] +(31) CometColumnarToRow [codegen id : 3] Input [2]: [ca_address_sk#20, ca_state#21] (32) BroadcastExchange @@ -214,7 +215,7 @@ ReadSchema: struct -(46) CometFilter +(47) 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 +(48) 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] +(49) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] -(49) BroadcastExchange +(50) 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 0b55e23ab6..82c1f313df 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,71 @@ 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 + CometColumnarToRow + InputAdapter + CometColumnarExchange [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] + CometColumnarToRow + 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) + CometColumnarToRow + 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 - 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] + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + 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 #5 - WholeStageCodegen (1) - ColumnarToRow + BroadcastExchange #6 + WholeStageCodegen (2) + CometColumnarToRow 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] + 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) + CometColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + CometColumnarToRow 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 #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 c5bb669870..edc4b014bc 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometHashAggregate (36) +- CometExchange (35) @@ -215,14 +215,14 @@ Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_de 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] +(38) CometColumnarToRow [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 ===== Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999 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] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (43) BroadcastExchange 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 5ebac30264..18c0a90bd8 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -21,7 +21,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_qoy] 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 e58a49d587..97effe35f3 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,7 +2,7 @@ TakeOrderedAndProject (42) +- * Project (41) +- Window (40) - +- * ColumnarToRow (39) + +- * CometColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -220,7 +220,7 @@ Arguments: hashpartitioning(lochierarchy#33, _w0#45, 5), ENSURE_REQUIREMENTS, Co 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) ColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#30, i_category#11, i_class#10, lochierarchy#33, _w0#45] (40) Window @@ -239,7 +239,7 @@ Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (47) -+- * ColumnarToRow (46) ++- * CometColumnarToRow (46) +- CometProject (45) +- CometFilter (44) +- CometScan parquet spark_catalog.default.date_dim (43) @@ -260,7 +260,7 @@ Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(46) ColumnarToRow [codegen id : 1] +(46) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] (47) BroadcastExchange 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 6eb469dae6..9a3655e498 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter Window [gross_margin,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -26,7 +26,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_year] 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 41c364a3f6..51357d654d 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_ 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#20, i_brand#21, s_store_name#22, s_compan 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [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] (34) Window @@ -225,7 +225,7 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] Arguments: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#16] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#16 - avg_monthly_sales#19) ASC NULLS FIRST, d_moy#11 Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 a18f68d837..6355321c9b 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 685f048dea..281d422ff4 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,82 @@ == Physical Plan == -TakeOrderedAndProject (77) -+- * HashAggregate (76) - +- Exchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * ColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (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) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (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) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (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) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (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) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (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) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (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) CometScan parquet spark_catalog.default.web_sales @@ -172,7 +173,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] 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) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] (22) Window @@ -273,7 +274,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] 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) ColumnarToRow [codegen id : 4] +(44) CometColumnarToRow [codegen id : 4] Input [3]: [item#47, return_ratio#48, currency_ratio#49] (45) Window @@ -374,7 +375,7 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] 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) ColumnarToRow [codegen id : 7] +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] (68) Window @@ -406,50 +407,51 @@ Functions: [] Aggregate Attributes: [] Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) Exchange +(75) 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, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(76) HashAggregate [codegen id : 11] +(76) 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#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(77) TakeOrderedAndProject +(77) 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] + +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: 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] ===== 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 (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan parquet spark_catalog.default.date_dim (79) -(78) CometScan parquet spark_catalog.default.date_dim +(79) CometScan 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 +(80) 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 +(81) 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] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(82) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] 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/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index d98b2b0a66..21827c75d8 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,109 +1,110 @@ -TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] - WholeStageCodegen (11) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 - 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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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] - CometExchange #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))] - CometExchange [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 +WholeStageCodegen (11) + CometColumnarToRow + 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 + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + 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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #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))] + CometExchange [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 3211e46f6d..efa9fb0bdf 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,76 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Filter (71) - +- * HashAggregate (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- Window (61) - : +- * Sort (60) - : +- Exchange (59) - : +- * Project (58) - : +- * Filter (57) - : +- * SortMergeJoin FullOuter (56) - : :- * Sort (29) - : : +- Exchange (28) - : : +- * HashAggregate (27) - : : +- Exchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * ColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (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) - : +- * Sort (55) - : +- Exchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (42) - : : +- Window (41) - : : +- * ColumnarToRow (40) - : : +- CometSort (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometExchange (36) - : : +- CometHashAggregate (35) - : : +- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometFilter (31) - : : : +- CometScan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (32) - : +- BroadcastExchange (48) - : +- * Project (47) - : +- Window (46) - : +- * ColumnarToRow (45) - : +- CometSort (44) - : +- ReusedExchange (43) - +- BroadcastExchange (66) - +- * Project (65) - +- Window (64) - +- * Sort (63) - +- ReusedExchange (62) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (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) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan parquet spark_catalog.default.web_sales @@ -135,7 +139,7 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) ColumnarToRow [codegen id : 1] +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] (15) Window @@ -153,7 +157,7 @@ Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] 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) ColumnarToRow [codegen id : 2] +(19) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] (20) Window @@ -185,26 +189,29 @@ 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) Exchange +(26) 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, [plan_id=4] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) HashAggregate [codegen id : 5] +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(28) 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] -(28) Exchange +(29) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) Sort [codegen id : 6] +(30) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(30) CometScan parquet spark_catalog.default.store_sales +(31) CometScan 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 [] @@ -212,222 +219,230 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Condition : isnotnull(ss_item_sk#23) -(32) ReusedExchange [Reuses operator id: 6] +(33) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#27, d_date#28] -(33) CometBroadcastHashJoin +(34) 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 -(34) CometProject +(35) 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] -(35) CometHashAggregate +(36) 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))] -(36) CometExchange +(37) CometExchange Input [3]: [ss_item_sk#23, d_date#28, sum#29] Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(37) CometHashAggregate +(38) 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))] -(38) CometExchange +(39) CometExchange Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort +(40) 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] -(40) ColumnarToRow [codegen id : 7] +(41) CometColumnarToRow [codegen id : 6] Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(41) Window +(42) 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] -(42) Project [codegen id : 10] +(43) 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] -(43) ReusedExchange [Reuses operator id: 38] +(44) ReusedExchange [Reuses operator id: 39] Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(44) CometSort +(45) 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] -(45) ColumnarToRow [codegen id : 8] +(46) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(46) Window +(47) 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] -(47) Project [codegen id : 9] +(48) 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] -(48) BroadcastExchange +(49) BroadcastExchange Input [3]: [item_sk#36, sumss#37, rk#35] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(49) BroadcastHashJoin [codegen id : 10] +(50) BroadcastHashJoin [codegen id : 9] Left keys [1]: [item_sk#30] Right keys [1]: [item_sk#36] Join type: Inner Join condition: (rk#32 >= rk#35) -(50) Project [codegen id : 10] +(51) 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] -(51) HashAggregate [codegen id : 10] +(52) 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] -(52) Exchange +(53) 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, [plan_id=9] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(53) HashAggregate [codegen id : 11] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) 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] -(54) Exchange +(56) CometColumnarExchange Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(55) Sort [codegen id : 12] +(57) CometSort Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(56) SortMergeJoin [codegen id : 13] -Left keys [2]: [item_sk#9, d_date#6] -Right keys [2]: [item_sk#30, d_date#28] -Join type: FullOuter -Join condition: None +(58) 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 -(57) Filter [codegen id : 13] +(59) 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) -(58) Project [codegen id : 13] -Output [4]: [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] +(60) 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] + +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(59) Exchange +(62) CometSort Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=11] +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] -(60) Sort [codegen id : 14] +(63) CometColumnarToRow [codegen id : 11] Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 -(61) Window +(64) 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] -(62) ReusedExchange [Reuses operator id: 59] +(65) ReusedExchange [Reuses operator id: 61] Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(63) Sort [codegen id : 28] +(66) CometSort Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 +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] -(64) Window +(67) CometColumnarToRow [codegen id : 22] +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#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] -(65) Project [codegen id : 29] +(69) 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] -(66) BroadcastExchange +(70) 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] -(67) BroadcastHashJoin [codegen id : 30] +(71) BroadcastHashJoin [codegen id : 24] Left keys [1]: [item_sk#44] Right keys [1]: [item_sk#50] Join type: Inner Join condition: (rk#48 >= rk#49) -(68) Project [codegen id : 30] +(72) 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] -(69) HashAggregate [codegen id : 30] +(73) 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] -(70) HashAggregate [codegen id : 30] +(74) 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] -(71) Filter [codegen id : 30] +(75) 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)) -(72) TakeOrderedAndProject +(76) 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 (77) -+- * ColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan parquet spark_catalog.default.date_dim (73) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) -(73) CometScan parquet spark_catalog.default.date_dim +(77) CometScan 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 -(74) CometFilter +(78) 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)) -(75) CometProject +(79) 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] -(76) ColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(77) BroadcastExchange +(81) 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 = 30 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 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 f8e1ba2353..51540b97fd 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 (30) + 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,113 +7,110 @@ 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 (14) - Sort [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow InputAdapter - Exchange [item_sk] #1 - WholeStageCodegen (13) - 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 (6) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumws] #3 - 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 (1) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] - CometExchange [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 (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - InputAdapter - WholeStageCodegen (12) - Sort [item_sk,d_date] - InputAdapter - Exchange [item_sk,d_date] #9 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - InputAdapter - Exchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (10) - 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 (7) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] - CometExchange [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 (9) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (8) - ColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [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 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + 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 (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [item_sk,d_date,sumws,ws_item_sk,sum,sum(UnscaledValue(ws_sales_price))] + CometExchange [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) + CometColumnarToRow + 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 (3) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + 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 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + 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 (6) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [item_sk,d_date,sumss,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometExchange [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 (8) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (29) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (28) - Sort [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow 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 4b3a3c576e..84c04cfce2 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject (47) : : +- Window (26) : : +- * Filter (25) : : +- Window (24) - : : +- * ColumnarToRow (23) + : : +- * CometColumnarToRow (23) : : +- CometSort (22) : : +- CometExchange (21) : : +- CometHashAggregate (20) @@ -35,7 +35,7 @@ TakeOrderedAndProject (47) : +- BroadcastExchange (36) : +- * Project (35) : +- Window (34) - : +- * ColumnarToRow (33) + : +- * CometColumnarToRow (33) : +- CometSort (32) : +- CometExchange (31) : +- CometHashAggregate (30) @@ -43,7 +43,7 @@ TakeOrderedAndProject (47) +- BroadcastExchange (44) +- * Project (43) +- Window (42) - +- * ColumnarToRow (41) + +- * CometColumnarToRow (41) +- CometSort (40) +- ReusedExchange (39) @@ -154,7 +154,7 @@ Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQU 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] +(23) CometColumnarToRow [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 @@ -193,7 +193,7 @@ Arguments: hashpartitioning(i_category#19, i_brand#20, cc_name#21, 5), ENSURE_RE 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] -(33) ColumnarToRow [codegen id : 3] +(33) CometColumnarToRow [codegen id : 3] Input [6]: [i_category#19, i_brand#20, cc_name#21, d_year#22, d_moy#23, sum_sales#15] (34) Window @@ -225,7 +225,7 @@ Output [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sal Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] Arguments: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15], [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] -(41) ColumnarToRow [codegen id : 5] +(41) CometColumnarToRow [codegen id : 5] Input [6]: [i_category#28, i_brand#29, cc_name#30, d_year#31, d_moy#32, sum_sales#15] (42) Window @@ -258,7 +258,7 @@ Arguments: 100, [(sum_sales#15 - avg_monthly_sales#18) ASC NULLS FIRST, d_year#1 Subquery:1 Hosting operator id = 3 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (51) -+- * ColumnarToRow (50) ++- * CometColumnarToRow (50) +- CometFilter (49) +- CometScan parquet spark_catalog.default.date_dim (48) @@ -274,7 +274,7 @@ ReadSchema: struct 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)) -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (51) BroadcastExchange 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 3ea3165fc7..e9ee530494 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 @@ -13,7 +13,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 (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 @@ -34,7 +34,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year,d_moy] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] @@ -51,7 +51,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 (3) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 @@ -64,7 +64,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 (5) - ColumnarToRow + CometColumnarToRow InputAdapter 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/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 564222abff..47368bc7dd 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,93 @@ == Physical Plan == -TakeOrderedAndProject (83) -+- * HashAggregate (82) - +- Exchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * HashAggregate (68) - : +- Exchange (67) - : +- * HashAggregate (66) - : +- Union (65) - : :- * HashAggregate (22) - : : +- * ColumnarToRow (21) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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) +* CometColumnarToRow (89) ++- CometTakeOrderedAndProject (88) + +- CometHashAggregate (87) + +- CometColumnarExchange (86) + +- * HashAggregate (85) + +- Union (84) + :- * HashAggregate (69) + : +- * CometColumnarToRow (68) + : +- CometColumnarExchange (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (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) + : : +- * CometColumnarToRow (40) + : : +- CometExchange (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) + : +- * CometColumnarToRow (63) + : +- CometExchange (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 (76) + : +- * CometColumnarToRow (75) + : +- CometColumnarExchange (74) + : +- * HashAggregate (73) + : +- * HashAggregate (72) + : +- * CometColumnarToRow (71) + : +- ReusedExchange (70) + +- * HashAggregate (83) + +- * CometColumnarToRow (82) + +- CometColumnarExchange (81) + +- * HashAggregate (80) + +- * HashAggregate (79) + +- * CometColumnarToRow (78) + +- ReusedExchange (77) (1) CometScan parquet spark_catalog.default.store_sales @@ -181,7 +187,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(21) ColumnarToRow [codegen id : 1] +(21) CometColumnarToRow [codegen id : 1] Input [5]: [s_store_id#25, sum#26, sum#27, sum#28, sum#29] (22) HashAggregate [codegen id : 1] @@ -272,7 +278,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#46)), partial_sum(Unscaled 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, CometNativeShuffle, [plan_id=2] -(40) ColumnarToRow [codegen id : 2] +(40) CometColumnarToRow [codegen id : 2] Input [5]: [cp_catalog_page_id#62, sum#63, sum#64, sum#65, sum#66] (41) HashAggregate [codegen id : 2] @@ -382,7 +388,7 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#83)), partial_sum(Unscaled Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(63) ColumnarToRow [codegen id : 3] +(63) CometColumnarToRow [codegen id : 3] Input [5]: [web_site_id#104, sum#105, sum#106, sum#107, sum#108] (64) HashAggregate [codegen id : 3] @@ -401,126 +407,142 @@ Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(prof 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) 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, [plan_id=4] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(68) HashAggregate [codegen id : 5] +(68) CometColumnarToRow [codegen id : 5] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(69) 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] +(70) ReusedExchange [Reuses operator id: 67] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(70) HashAggregate [codegen id : 10] +(71) CometColumnarToRow [codegen id : 10] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(72) 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 : 10] +(73) 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 +(74) CometColumnarExchange 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] +Arguments: hashpartitioning(channel#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(73) HashAggregate [codegen id : 11] +(75) CometColumnarToRow [codegen id : 11] +Input [7]: [channel#34, sum#145, isEmpty#146, sum#147, isEmpty#148, sum#149, isEmpty#150] + +(76) 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] +(77) ReusedExchange [Reuses operator id: 67] Output [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -(75) HashAggregate [codegen id : 16] +(78) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#34, id#35, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(79) 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 : 16] +(80) 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 +(81) CometColumnarExchange +Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(82) CometColumnarToRow [codegen id : 17] Input [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(78) HashAggregate [codegen id : 17] +(83) 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 +(84) Union -(80) HashAggregate [codegen id : 18] +(85) 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 +(86) 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, [plan_id=7] +Arguments: hashpartitioning(channel#34, id#35, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(82) HashAggregate [codegen id : 19] +(87) 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 +(88) 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] + +(89) CometColumnarToRow [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 (94) ++- * CometColumnarToRow (93) + +- CometProject (92) + +- CometFilter (91) + +- CometScan parquet spark_catalog.default.date_dim (90) -(84) CometScan parquet spark_catalog.default.date_dim +(90) CometScan 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 +(91) 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 +(92) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(87) ColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(88) BroadcastExchange +(94) 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 751b3dc240..712795e719 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,118 +1,124 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - 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)),channel,id,sales,returns,profit,sum,sum,sum,sum] - ColumnarToRow - InputAdapter - CometExchange [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 - CometExchange [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) - 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 - CometExchange [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] - InputAdapter - Exchange [channel] #12 - 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] - 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] - InputAdapter - Exchange #13 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + 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)),channel,id,sales,returns,profit,sum,sum,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [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) + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometExchange [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) + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #12 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #13 + 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] + CometColumnarToRow + 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 54cbf8326b..77ad43a121 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (38) +* CometColumnarToRow (38) +- CometTakeOrderedAndProject (37) +- CometFilter (36) +- CometHashAggregate (35) @@ -215,14 +215,14 @@ Condition : (cnt#23 >= 10) 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] +(38) CometColumnarToRow [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 (43) -+- * ColumnarToRow (42) ++- * CometColumnarToRow (42) +- CometProject (41) +- CometFilter (40) +- CometScan parquet spark_catalog.default.date_dim (39) @@ -243,7 +243,7 @@ Condition : ((isnotnull(d_month_seq#10) AND (d_month_seq#10 = Subquery scalar-su Input [2]: [d_date_sk#9, d_month_seq#10] Arguments: [d_date_sk#9], [d_date_sk#9] -(42) ColumnarToRow [codegen id : 1] +(42) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#9] (43) BroadcastExchange @@ -251,7 +251,7 @@ Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] Subquery:2 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* ColumnarToRow (50) +* CometColumnarToRow (50) +- CometHashAggregate (49) +- CometExchange (48) +- CometHashAggregate (47) @@ -289,7 +289,7 @@ Input [1]: [d_month_seq#24] Keys [1]: [d_month_seq#24] Functions: [] -(50) ColumnarToRow [codegen id : 1] +(50) CometColumnarToRow [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 580f668ea8..266060bf09 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] @@ -25,13 +25,13 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] Subquery #2 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometHashAggregate [d_month_seq] CometExchange [d_month_seq] #5 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 4d8ac469c3..7252446de1 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (173) +* CometColumnarToRow (173) +- CometSort (172) +- CometColumnarExchange (171) +- CometProject (170) @@ -938,14 +938,14 @@ Arguments: rangepartitioning(product_name#76 ASC NULLS FIRST, store_name#78 ASC 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] -(173) ColumnarToRow [codegen id : 1] +(173) CometColumnarToRow [codegen id : 1] 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 (177) -+- * ColumnarToRow (176) ++- * CometColumnarToRow (176) +- CometFilter (175) +- CometScan parquet spark_catalog.default.date_dim (174) @@ -961,7 +961,7 @@ ReadSchema: struct 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)) -(176) ColumnarToRow [codegen id : 1] +(176) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#32, d_year#33] (177) BroadcastExchange @@ -970,7 +970,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#104 IN dynamicpruning#105 BroadcastExchange (181) -+- * ColumnarToRow (180) ++- * CometColumnarToRow (180) +- CometFilter (179) +- CometScan parquet spark_catalog.default.date_dim (178) @@ -986,7 +986,7 @@ ReadSchema: struct 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)) -(180) ColumnarToRow [codegen id : 1] +(180) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#117, d_year#118] (181) BroadcastExchange 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 f38a6afca6..f4df8e5826 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -51,7 +51,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -158,7 +158,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #22 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 379144482c..10156503f1 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,7 +2,7 @@ TakeOrderedAndProject (68) +- * Filter (67) +- Window (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSort (64) +- CometExchange (63) +- CometUnion (62) @@ -366,7 +366,7 @@ Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeS 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] +(65) CometColumnarToRow [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 @@ -385,7 +385,7 @@ Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_br Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (73) -+- * ColumnarToRow (72) ++- * CometColumnarToRow (72) +- CometProject (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -406,7 +406,7 @@ Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_mont 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] -(72) ColumnarToRow [codegen id : 1] +(72) CometColumnarToRow [codegen id : 1] Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] (73) BroadcastExchange 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 3216530ac2..9410250201 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Window [sumsales,i_category] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometExchange [i_category] #1 @@ -23,7 +23,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_year,d_moy,d_qoy] CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] 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 4a04020a03..5032fd2cd4 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,67 @@ == 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) - : +- * ColumnarToRow (27) - : +- CometSort (26) - : +- CometHashAggregate (25) - : +- CometExchange (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 (63) ++- * Project (62) + +- Window (61) + +- * CometColumnarToRow (60) + +- CometSort (59) + +- CometExchange (58) + +- CometHashAggregate (57) + +- CometColumnarExchange (56) + +- * HashAggregate (55) + +- Union (54) + :- * HashAggregate (39) + : +- * CometColumnarToRow (38) + : +- CometColumnarExchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * CometColumnarToRow (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) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (31) + : +- * Project (30) + : +- * Filter (29) + : +- Window (28) + : +- * CometColumnarToRow (27) + : +- CometSort (26) + : +- CometHashAggregate (25) + : +- CometExchange (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 (46) + : +- * CometColumnarToRow (45) + : +- CometColumnarExchange (44) + : +- * HashAggregate (43) + : +- * HashAggregate (42) + : +- * CometColumnarToRow (41) + : +- ReusedExchange (40) + +- * HashAggregate (53) + +- * CometColumnarToRow (52) + +- CometColumnarExchange (51) + +- * HashAggregate (50) + +- * HashAggregate (49) + +- * CometColumnarToRow (48) + +- ReusedExchange (47) (1) CometScan parquet spark_catalog.default.store_sales @@ -98,7 +104,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 : 4] +(9) CometColumnarToRow [codegen id : 4] Input [2]: [ss_store_sk#1, ss_net_profit#2] (10) CometScan parquet spark_catalog.default.store @@ -112,7 +118,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 : 3] +(12) CometColumnarToRow [codegen id : 3] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (13) CometScan parquet spark_catalog.default.store_sales @@ -181,7 +187,7 @@ Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] 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] +(27) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#15, _w0#18, s_state#15] (28) Window @@ -227,142 +233,158 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum#20] Results [3]: [s_state#9, s_county#8, sum#21] -(37) Exchange +(37) CometColumnarExchange Input [3]: [s_state#9, s_county#8, sum#21] -Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Arguments: hashpartitioning(s_state#9, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(38) HashAggregate [codegen id : 5] +(38) CometColumnarToRow [codegen id : 5] +Input [3]: [s_state#9, s_county#8, sum#21] + +(39) 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))#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] -(39) ReusedExchange [Reuses operator id: 37] +(40) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#9, s_county#8, sum#27] -(40) HashAggregate [codegen id : 10] +(41) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#9, s_county#8, sum#27] + +(42) 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))#22] Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#9] -(41) HashAggregate [codegen id : 10] +(43) HashAggregate [codegen id : 10] Input [2]: [total_sum#28, s_state#9] Keys [1]: [s_state#9] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#29, isEmpty#30] Results [3]: [s_state#9, sum#31, isEmpty#32] -(42) Exchange +(44) CometColumnarExchange Input [3]: [s_state#9, sum#31, isEmpty#32] -Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Arguments: hashpartitioning(s_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(43) HashAggregate [codegen id : 11] +(45) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#9, sum#31, isEmpty#32] + +(46) HashAggregate [codegen id : 11] Input [3]: [s_state#9, sum#31, isEmpty#32] Keys [1]: [s_state#9] 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] -(44) ReusedExchange [Reuses operator id: 37] +(47) ReusedExchange [Reuses operator id: 37] Output [3]: [s_state#9, s_county#8, sum#39] -(45) HashAggregate [codegen id : 16] +(48) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#9, s_county#8, sum#39] + +(49) 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))#22] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28] -(46) HashAggregate [codegen id : 16] +(50) HashAggregate [codegen id : 16] Input [1]: [total_sum#28] Keys: [] Functions [1]: [partial_sum(total_sum#28)] Aggregate Attributes [2]: [sum#40, isEmpty#41] Results [2]: [sum#42, isEmpty#43] -(47) Exchange +(51) CometColumnarExchange +Input [2]: [sum#42, isEmpty#43] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(52) CometColumnarToRow [codegen id : 17] Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] -(48) HashAggregate [codegen id : 17] +(53) HashAggregate [codegen id : 17] Input [2]: [sum#42, isEmpty#43] Keys: [] 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 +(54) Union -(50) HashAggregate [codegen id : 18] +(55) 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#23, s_state#9, s_county#8, g_state#24, g_county#25, lochierarchy#26] -(51) Exchange +(56) 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, [plan_id=7] +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] -(52) HashAggregate [codegen id : 19] +(57) 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#23, s_state#9, s_county#8, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#9 END AS _w0#51] -(53) Exchange +(58) CometExchange +Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] +Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(59) CometSort Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] -Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, [plan_id=8] +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 : 20] +(60) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#23, s_state#9, s_county#8, lochierarchy#26, _w0#51] -Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0 -(55) Window +(61) 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 : 21] +(62) 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 +(63) 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 (68) ++- * CometColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) -(58) CometScan parquet spark_catalog.default.date_dim +(64) CometScan 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 +(65) 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 +(66) 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] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(62) BroadcastExchange +(68) 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 a1d9ab5183..fc43e8ce05 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,29 +1,29 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (21) + WholeStageCodegen (20) Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (20) - Sort [lochierarchy,_w0,total_sum] + WholeStageCodegen (19) + CometColumnarToRow InputAdapter - Exchange [lochierarchy,_w0] #1 - WholeStageCodegen (19) - 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 (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [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 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,g_state,g_county,lochierarchy,sum] + CometColumnarToRow InputAdapter - Exchange [s_state,s_county] #3 + CometColumnarExchange [s_state,s_county] #3 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 + CometColumnarToRow InputAdapter CometProject [ss_store_sk,ss_net_profit] CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] @@ -32,7 +32,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] @@ -45,7 +45,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count BroadcastExchange #6 WholeStageCodegen (3) BroadcastHashJoin [s_state,s_state] - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [s_store_sk,s_county,s_state] CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] @@ -57,7 +57,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter Window [_w0,s_state] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [s_state,_w0] CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] @@ -74,21 +74,25 @@ 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 (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow InputAdapter - Exchange [s_state] #10 + CometColumnarExchange [s_state] #10 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] - 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] + CometColumnarToRow + 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] + CometColumnarToRow InputAdapter - Exchange #11 + CometColumnarExchange #11 WholeStageCodegen (16) 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 + CometColumnarToRow + 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 45f7e2e66d..4b97b311be 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (64) +* CometColumnarToRow (64) +- CometTakeOrderedAndProject (63) +- CometHashAggregate (62) +- CometExchange (61) @@ -364,14 +364,14 @@ Functions [1]: [count(1)] 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] -(64) ColumnarToRow [codegen id : 1] +(64) CometColumnarToRow [codegen id : 1] 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 (69) -+- * ColumnarToRow (68) ++- * CometColumnarToRow (68) +- CometProject (67) +- CometFilter (66) +- CometScan parquet spark_catalog.default.date_dim (65) @@ -392,7 +392,7 @@ Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_dat 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] -(68) ColumnarToRow [codegen id : 1] +(68) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] (69) BroadcastExchange 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 ccd1b66fa2..fe26a3f5cc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow 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)] @@ -32,7 +32,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk,d_date,d_week_seq] CometFilter [d_date_sk,d_date,d_week_seq,d_year] 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 12d5134a06..b1b1e31edb 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (68) +* CometColumnarToRow (68) +- CometTakeOrderedAndProject (67) +- CometProject (66) +- CometBroadcastHashJoin (65) @@ -388,14 +388,14 @@ Arguments: [customer_id#25, customer_first_name#26, customer_last_name#27], [cus 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] +(68) CometColumnarToRow [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 (72) -+- * ColumnarToRow (71) ++- * CometColumnarToRow (71) +- CometFilter (70) +- CometScan parquet spark_catalog.default.date_dim (69) @@ -411,7 +411,7 @@ ReadSchema: struct 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)) -(71) ColumnarToRow [codegen id : 1] +(71) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#9, d_year#10] (72) BroadcastExchange @@ -420,7 +420,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 BroadcastExchange (76) -+- * ColumnarToRow (75) ++- * CometColumnarToRow (75) +- CometFilter (74) +- CometScan parquet spark_catalog.default.date_dim (73) @@ -436,7 +436,7 @@ ReadSchema: struct 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)) -(75) ColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#22, d_year#23] (76) BroadcastExchange 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 631a82f1e3..9bde9ac863 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt @@ -1,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometProject [customer_id,customer_first_name,customer_last_name] @@ -23,7 +23,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -46,7 +46,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #8 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index 6789a22468..b8bba79fa5 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,5 +1,5 @@ == Physical Plan == -* ColumnarToRow (124) +* CometColumnarToRow (124) +- CometTakeOrderedAndProject (123) +- CometProject (122) +- CometSortMergeJoin (121) @@ -676,14 +676,14 @@ Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#1 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: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[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]), [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], 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] -(124) ColumnarToRow [codegen id : 1] +(124) CometColumnarToRow [codegen id : 1] 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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 BroadcastExchange (128) -+- * ColumnarToRow (127) ++- * CometColumnarToRow (127) +- CometFilter (126) +- CometScan parquet spark_catalog.default.date_dim (125) @@ -699,7 +699,7 @@ ReadSchema: struct 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)) -(127) ColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] (128) BroadcastExchange @@ -712,7 +712,7 @@ Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN d Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 BroadcastExchange (132) -+- * ColumnarToRow (131) ++- * CometColumnarToRow (131) +- CometFilter (130) +- CometScan parquet spark_catalog.default.date_dim (129) @@ -728,7 +728,7 @@ ReadSchema: struct 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)) -(131) ColumnarToRow [codegen id : 1] +(131) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#77, d_year#78] (132) BroadcastExchange 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 9e340130a5..5a8af58abc 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,5 +1,5 @@ WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] @@ -27,7 +27,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #5 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometFilter [d_date_sk,d_year] CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] @@ -102,7 +102,7 @@ WholeStageCodegen (1) SubqueryBroadcast [d_date_sk] #2 BroadcastExchange #17 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index dcbd98dbb8..2c7e678355 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,101 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (97) -+- * HashAggregate (96) - +- Exchange (95) - +- * HashAggregate (94) - +- Union (93) - :- * HashAggregate (82) - : +- Exchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * ColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (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) - : : +- CometExchange (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) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * ColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (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) - : : +- CometExchange (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) - : +- CometExchange (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 (87) - : +- Exchange (86) - : +- * HashAggregate (85) - : +- * HashAggregate (84) - : +- ReusedExchange (83) - +- * HashAggregate (92) - +- Exchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- ReusedExchange (88) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometColumnarExchange (100) + +- * HashAggregate (99) + +- Union (98) + :- * HashAggregate (83) + : +- * CometColumnarToRow (82) + : +- CometColumnarExchange (81) + : +- * HashAggregate (80) + : +- Union (79) + : :- * CometColumnarToRow (31) + : : +- CometProject (30) + : : +- CometBroadcastHashJoin (29) + : : :- CometHashAggregate (16) + : : : +- CometExchange (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) + : : +- CometExchange (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) + : : : +- * CometColumnarToRow (39) + : : : +- CometHashAggregate (38) + : : : +- CometExchange (37) + : : : +- CometHashAggregate (36) + : : : +- CometProject (35) + : : : +- CometBroadcastHashJoin (34) + : : : :- CometScan parquet spark_catalog.default.catalog_sales (32) + : : : +- ReusedExchange (33) + : : +- * CometColumnarToRow (48) + : : +- CometHashAggregate (47) + : : +- CometExchange (46) + : : +- CometHashAggregate (45) + : : +- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometScan parquet spark_catalog.default.catalog_returns (41) + : : +- ReusedExchange (42) + : +- * CometColumnarToRow (78) + : +- CometProject (77) + : +- CometBroadcastHashJoin (76) + : :- CometHashAggregate (63) + : : +- CometExchange (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) + : +- CometExchange (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 (90) + : +- * CometColumnarToRow (89) + : +- CometColumnarExchange (88) + : +- * HashAggregate (87) + : +- * HashAggregate (86) + : +- * CometColumnarToRow (85) + : +- ReusedExchange (84) + +- * HashAggregate (97) + +- * CometColumnarToRow (96) + +- CometColumnarExchange (95) + +- * HashAggregate (94) + +- * HashAggregate (93) + +- * CometColumnarToRow (92) + +- ReusedExchange (91) (1) CometScan parquet spark_catalog.default.store_sales @@ -239,7 +245,7 @@ Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight 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] -(31) ColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] (32) CometScan parquet spark_catalog.default.catalog_sales @@ -275,7 +281,7 @@ 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))] -(39) ColumnarToRow [codegen id : 2] +(39) CometColumnarToRow [codegen id : 2] Input [3]: [cs_call_center_sk#28, sales#36, profit#37] (40) BroadcastExchange @@ -315,7 +321,7 @@ Input [2]: [sum#43, sum#44] Keys: [] Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(48) ColumnarToRow +(48) CometColumnarToRow Input [2]: [returns#45, profit_loss#46] (49) BroadcastNestedLoopJoin [codegen id : 3] @@ -451,7 +457,7 @@ Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight 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] +(78) CometColumnarToRow [codegen id : 4] Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] (79) Union @@ -463,126 +469,142 @@ Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(prof 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) Exchange +(81) 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, [plan_id=8] +Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(82) HashAggregate [codegen id : 6] +(82) CometColumnarToRow [codegen id : 6] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(83) 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] -(83) ReusedExchange [Reuses operator id: 81] +(84) ReusedExchange [Reuses operator id: 81] Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -(84) HashAggregate [codegen id : 12] +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(86) 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] -(85) HashAggregate [codegen id : 12] +(87) 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] -(86) Exchange +(88) CometColumnarExchange Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] -(87) HashAggregate [codegen id : 13] +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(90) 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] -(88) ReusedExchange [Reuses operator id: 81] +(91) ReusedExchange [Reuses operator id: 81] Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -(89) HashAggregate [codegen id : 19] +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] + +(93) 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] -(90) HashAggregate [codegen id : 19] +(94) HashAggregate [codegen id : 19] Input [3]: [sales#94, returns#95, profit#96] Keys: [] 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] -(91) Exchange +(95) CometColumnarExchange +Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(96) CometColumnarToRow [codegen id : 20] Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] -(92) HashAggregate [codegen id : 20] +(97) HashAggregate [codegen id : 20] Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] Keys: [] 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] -(93) Union +(98) Union -(94) HashAggregate [codegen id : 21] +(99) 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#24, id#25, sales#91, returns#92, profit#93] -(95) Exchange +(100) 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, [plan_id=11] +Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(96) HashAggregate [codegen id : 22] +(101) 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#24, id#25, sales#91, returns#92, profit#93] -(97) TakeOrderedAndProject +(102) 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] + +(103) CometColumnarToRow [codegen id : 22] Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: 100, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [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 (102) -+- * ColumnarToRow (101) - +- CometProject (100) - +- CometFilter (99) - +- CometScan parquet spark_catalog.default.date_dim (98) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) -(98) CometScan parquet spark_catalog.default.date_dim +(104) CometScan 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 -(99) CometFilter +(105) 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)) -(100) CometProject +(106) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(101) ColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(102) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] 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 01cd2f085f..02f19882bd 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,135 +1,141 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (22) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - 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))] - CometExchange [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))] - CometExchange [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] - 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 (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - ColumnarToRow - InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [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 +WholeStageCodegen (22) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (5) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (1) + CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #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 - 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))] - CometExchange [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 + 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))] + CometExchange [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) + CometColumnarToRow + 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))] + CometExchange [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] + 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 (3) + Project [cs_call_center_sk,sales,returns,profit,profit_loss] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometExchange [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 + CometColumnarToRow + InputAdapter + CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometExchange #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] - 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))] - CometExchange [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] - 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 (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] - InputAdapter - Exchange [channel] #16 - 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] - 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] - InputAdapter - Exchange #17 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (4) + CometColumnarToRow + 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))] + CometExchange [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))] + CometExchange [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] + 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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #16 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #17 + 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] + CometColumnarToRow + 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 1637692970..9a697cc844 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,7 +1,7 @@ == Physical Plan == TakeOrderedAndProject (67) +- * Project (66) - +- * ColumnarToRow (65) + +- * CometColumnarToRow (65) +- CometSortMergeJoin (64) :- CometProject (43) : +- CometSortMergeJoin (42) @@ -360,7 +360,7 @@ Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss 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] +(65) CometColumnarToRow [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] @@ -375,7 +375,7 @@ Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 BroadcastExchange (71) -+- * ColumnarToRow (70) ++- * CometColumnarToRow (70) +- CometFilter (69) +- CometScan parquet spark_catalog.default.date_dim (68) @@ -391,7 +391,7 @@ ReadSchema: struct 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)) -(70) ColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] (71) BroadcastExchange 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 78d5fe1d7b..58ca9f30b8 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,7 +1,7 @@ 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 (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] - ColumnarToRow + CometColumnarToRow InputAdapter 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] @@ -22,7 +22,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #3 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter 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-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 442335b6dc..11dc89f090 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,119 +1,125 @@ == Physical Plan == -TakeOrderedAndProject (115) -+- * HashAggregate (114) - +- Exchange (113) - +- * HashAggregate (112) - +- Union (111) - :- * HashAggregate (100) - : +- Exchange (99) - : +- * HashAggregate (98) - : +- Union (97) - : :- * HashAggregate (38) - : : +- * ColumnarToRow (37) - : : +- CometExchange (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) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (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) - : : +- CometExchange (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) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (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) - : +- CometExchange (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) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (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 (105) - : +- Exchange (104) - : +- * HashAggregate (103) - : +- * HashAggregate (102) - : +- ReusedExchange (101) - +- * HashAggregate (110) - +- Exchange (109) - +- * HashAggregate (108) - +- * HashAggregate (107) - +- ReusedExchange (106) +* CometColumnarToRow (121) ++- CometTakeOrderedAndProject (120) + +- CometHashAggregate (119) + +- CometColumnarExchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (101) + : +- * CometColumnarToRow (100) + : +- CometColumnarExchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * HashAggregate (38) + : : +- * CometColumnarToRow (37) + : : +- CometExchange (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) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (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) + : : +- * CometColumnarToRow (66) + : : +- CometExchange (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) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (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) + : +- * CometColumnarToRow (95) + : +- CometExchange (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) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (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 (108) + : +- * CometColumnarToRow (107) + : +- CometColumnarExchange (106) + : +- * HashAggregate (105) + : +- * HashAggregate (104) + : +- * CometColumnarToRow (103) + : +- ReusedExchange (102) + +- * HashAggregate (115) + +- * CometColumnarToRow (114) + +- CometColumnarExchange (113) + +- * HashAggregate (112) + +- * HashAggregate (111) + +- * CometColumnarToRow (110) + +- ReusedExchange (109) (1) CometScan parquet spark_catalog.default.store_sales @@ -285,7 +291,7 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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, CometNativeShuffle, [plan_id=3] -(37) ColumnarToRow [codegen id : 1] +(37) CometColumnarToRow [codegen id : 1] Input [6]: [s_store_id#17, sum#22, sum#23, isEmpty#24, sum#25, isEmpty#26] (38) HashAggregate [codegen id : 1] @@ -416,7 +422,7 @@ Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#39)), partial_sum(c 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, CometNativeShuffle, [plan_id=6] -(66) ColumnarToRow [codegen id : 2] +(66) CometColumnarToRow [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] @@ -547,7 +553,7 @@ Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(c 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, CometNativeShuffle, [plan_id=9] -(95) ColumnarToRow [codegen id : 3] +(95) CometColumnarToRow [codegen id : 3] Input [6]: [web_site_id#81, sum#84, sum#85, isEmpty#86, sum#87, isEmpty#88] (96) HashAggregate [codegen id : 3] @@ -566,126 +572,142 @@ Functions [3]: [partial_sum(sales#32), partial_sum(returns#33), partial_sum(prof 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) Exchange +(99) 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, [plan_id=10] +Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(100) HashAggregate [codegen id : 5] +(100) CometColumnarToRow [codegen id : 5] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(101) 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] -(101) ReusedExchange [Reuses operator id: 99] +(102) ReusedExchange [Reuses operator id: 99] Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(102) HashAggregate [codegen id : 10] +(103) CometColumnarToRow [codegen id : 10] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(104) 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] -(103) HashAggregate [codegen id : 10] +(105) 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] -(104) Exchange +(106) CometColumnarExchange Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] -(105) HashAggregate [codegen id : 11] +(107) CometColumnarToRow [codegen id : 11] +Input [7]: [channel#30, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] + +(108) 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] -(106) ReusedExchange [Reuses operator id: 99] +(109) ReusedExchange [Reuses operator id: 99] Output [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -(107) HashAggregate [codegen id : 16] +(110) CometColumnarToRow [codegen id : 16] +Input [8]: [channel#30, id#31, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] + +(111) 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] -(108) HashAggregate [codegen id : 16] +(112) HashAggregate [codegen id : 16] Input [3]: [sales#115, returns#116, profit#117] Keys: [] 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] -(109) Exchange +(113) CometColumnarExchange +Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] + +(114) CometColumnarToRow [codegen id : 17] Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] -(110) HashAggregate [codegen id : 17] +(115) HashAggregate [codegen id : 17] Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys: [] 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] -(111) Union +(116) Union -(112) HashAggregate [codegen id : 18] +(117) 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#30, id#31, sales#112, returns#113, profit#114] -(113) Exchange +(118) 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, [plan_id=13] +Arguments: hashpartitioning(channel#30, id#31, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] -(114) HashAggregate [codegen id : 19] +(119) 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#30, id#31, sales#112, returns#113, profit#114] -(115) TakeOrderedAndProject +(120) 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] + +(121) CometColumnarToRow [codegen id : 19] Input [5]: [channel#30, id#31, sales#112, returns#113, profit#114] -Arguments: 100, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [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 (120) -+- * ColumnarToRow (119) - +- CometProject (118) - +- CometFilter (117) - +- CometScan parquet spark_catalog.default.date_dim (116) +BroadcastExchange (126) ++- * CometColumnarToRow (125) + +- CometProject (124) + +- CometFilter (123) + +- CometScan parquet spark_catalog.default.date_dim (122) -(116) CometScan parquet spark_catalog.default.date_dim +(122) CometScan 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 -(117) CometFilter +(123) 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)) -(118) CometProject +(124) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(119) ColumnarToRow [codegen id : 1] +(125) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(120) BroadcastExchange +(126) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] 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 aa61bbd579..582b39e19b 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,147 +1,153 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (19) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Exchange [channel,id,sales,returns,profit] #1 - 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] - InputAdapter - Exchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - 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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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 - CometExchange [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] - CometExchange [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] - CometExchange [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 (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] - InputAdapter - Exchange [channel] #19 - 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] - 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] - InputAdapter - Exchange #20 - 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] - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [channel,id,sales,returns,profit] + CometColumnarExchange [channel,id,sales,returns,profit] #1 + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel,id] #2 + WholeStageCodegen (4) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + 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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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) + CometColumnarToRow + 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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometExchange [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] + CometExchange [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] + CometExchange [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 (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] + CometColumnarToRow + InputAdapter + CometColumnarExchange [channel] #19 + 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] + CometColumnarToRow + 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] + CometColumnarToRow + InputAdapter + CometColumnarExchange #20 + 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] + CometColumnarToRow + 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 32e6981799..1817811aa1 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,7 +2,7 @@ TakeOrderedAndProject (36) +- * Project (35) +- Window (34) - +- * ColumnarToRow (33) + +- * CometColumnarToRow (33) +- CometSort (32) +- CometExchange (31) +- CometHashAggregate (30) @@ -186,7 +186,7 @@ Arguments: hashpartitioning(lochierarchy#21, _w0#33, 5), ENSURE_REQUIREMENTS, Co 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) ColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#18, i_category#9, i_class#8, lochierarchy#21, _w0#33] (34) Window @@ -205,7 +205,7 @@ Arguments: 100, [lochierarchy#21 DESC NULLS LAST, CASE WHEN (lochierarchy#21 = 0 Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 BroadcastExchange (41) -+- * ColumnarToRow (40) ++- * CometColumnarToRow (40) +- CometProject (39) +- CometFilter (38) +- CometScan parquet spark_catalog.default.date_dim (37) @@ -226,7 +226,7 @@ Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_mont Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(40) ColumnarToRow [codegen id : 1] +(40) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] (41) BroadcastExchange 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 ae161e5ae2..d2ed714b7e 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter Window [total_sum,lochierarchy,_w0] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0] CometExchange [lochierarchy,_w0] #1 @@ -24,7 +24,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_month_seq] 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 4d870a8b80..7bcf62d617 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,10 +1,10 @@ == Physical Plan == -* ColumnarToRow (24) +* CometColumnarToRow (24) +- CometSort (23) +- CometColumnarExchange (22) +- * Project (21) +- Window (20) - +- * ColumnarToRow (19) + +- * CometColumnarToRow (19) +- CometSort (18) +- CometExchange (17) +- CometHashAggregate (16) @@ -111,7 +111,7 @@ Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, CometNativeShuff 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) ColumnarToRow [codegen id : 1] +(19) CometColumnarToRow [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 @@ -130,14 +130,14 @@ Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS 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] -(24) ColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [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 (29) -+- * ColumnarToRow (28) ++- * CometColumnarToRow (28) +- CometProject (27) +- CometFilter (26) +- CometScan parquet spark_catalog.default.date_dim (25) @@ -158,7 +158,7 @@ Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 1999-02-22)) AND (d_date#1 Input [2]: [d_date_sk#11, d_date#12] Arguments: [d_date_sk#11], [d_date_sk#11] -(28) ColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] (29) BroadcastExchange 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 2b7813a707..6509498640 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,5 +1,5 @@ WholeStageCodegen (3) - ColumnarToRow + CometColumnarToRow InputAdapter 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 @@ -8,7 +8,7 @@ WholeStageCodegen (3) InputAdapter Window [_w0,i_class] WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 @@ -24,7 +24,7 @@ WholeStageCodegen (3) SubqueryBroadcast [d_date_sk] #1 BroadcastExchange #4 WholeStageCodegen (1) - ColumnarToRow + CometColumnarToRow InputAdapter CometProject [d_date_sk] CometFilter [d_date_sk,d_date] diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index f8c1a8b095..27d8e2357f 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -838,8 +838,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateTimestamps(), DataTypes.DoubleType) } - test("cast TimestampType to DecimalType(10,2)") { - castTest(generateTimestamps(), DataTypes.TimestampType) + ignore("cast TimestampType to DecimalType(10,2)") { + // https://github.com/apache/datafusion-comet/issues/1280 + // Native cast invoked for unsupported cast from Timestamp(Microsecond, Some("Etc/UTC")) to Decimal128(10, 2) + castTest(generateTimestamps(), DataTypes.createDecimalType(10, 2)) } test("cast TimestampType to StringType") { diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 5653eee019..30f485cff1 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -28,8 +28,8 @@ import scala.util.Random import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps -import org.apache.spark.sql.comet.CometProjectExec -import org.apache.spark.sql.execution.{ColumnarToRowExec, InputAdapter, ProjectExec, WholeStageCodegenExec} +import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} +import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -749,7 +749,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { val project = cometPlan .asInstanceOf[WholeStageCodegenExec] .child - .asInstanceOf[ColumnarToRowExec] + .asInstanceOf[CometColumnarToRowExec] .child .asInstanceOf[InputAdapter] .child @@ -1364,8 +1364,14 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { sql(s"create table $table(id int, name varchar(20)) using parquet") sql( s"insert into $table values(1, 'james smith'), (2, 'michael rose'), " + - "(3, 'robert williams'), (4, 'rames rose'), (5, 'james smith')") - checkSparkAnswerAndOperator(s"SELECT initcap(name) FROM $table") + "(3, 'robert williams'), (4, 'rames rose'), (5, 'james smith'), " + + "(6, 'robert rose-smith'), (7, 'james ähtäri')") + if (CometConf.COMET_EXEC_INITCAP_ENABLED.get()) { + // TODO: remove this if clause https://github.com/apache/datafusion-comet/issues/1052 + checkSparkAnswerAndOperator(s"SELECT initcap(name) FROM $table") + } else { + checkSparkAnswer(s"SELECT initcap(name) FROM $table") + } } } } @@ -2284,52 +2290,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - ignore("read map[int, int] from parquet") { - withTempPath { dir => - // create input file with Comet disabled - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - val df = spark - .range(5) - // Spark does not allow null as a key but does allow null as a - // value, and the entire map be null - .select( - when(col("id") > 1, map(col("id"), when(col("id") > 2, col("id")))).alias("map1")) - df.write.parquet(dir.toString()) - } - - Seq("", "parquet").foreach { v1List => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { - val df = spark.read.parquet(dir.toString()) - checkSparkAnswerAndOperator(df.select("map1")) - checkSparkAnswerAndOperator(df.select(map_keys(col("map1")))) - checkSparkAnswerAndOperator(df.select(map_values(col("map1")))) - } - } - } - } - - ignore("read array[int] from parquet") { - withTempPath { dir => - // create input file with Comet disabled - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - val df = spark - .range(5) - // Spark does not allow null as a key but does allow null as a - // value, and the entire map be null - .select(when(col("id") > 1, sequence(lit(0), col("id") * 2)).alias("array1")) - df.write.parquet(dir.toString()) - } - - Seq("", "parquet").foreach { v1List => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { - val df = spark.read.parquet(dir.toString()) - checkSparkAnswerAndOperator(df.select("array1")) - checkSparkAnswerAndOperator(df.select(element_at(col("array1"), lit(1)))) - } - } - } - } - test("get_struct_field with DataFusion ParquetExec - simple case") { withTempPath { dir => // create input file with Comet disabled @@ -2429,6 +2389,52 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + ignore("read map[int, int] from parquet") { + withTempPath { dir => +// create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) +// Spark does not allow null as a key but does allow null as a +// value, and the entire map be null + .select( + when(col("id") > 1, map(col("id"), when(col("id") > 2, col("id")))).alias("map1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("map1")) + checkSparkAnswerAndOperator(df.select(map_keys(col("map1")))) + checkSparkAnswerAndOperator(df.select(map_values(col("map1")))) + } + } + } + } + + ignore("read array[int] from parquet") { + withTempPath { dir => +// create input file with Comet disabled + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark + .range(5) +// Spark does not allow null as a key but does allow null as a +// value, and the entire map be null + .select(when(col("id") > 1, sequence(lit(0), col("id") * 2)).alias("array1")) + df.write.parquet(dir.toString()) + } + + Seq("", "parquet").foreach { v1List => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { + val df = spark.read.parquet(dir.toString()) + checkSparkAnswerAndOperator(df.select("array1")) + checkSparkAnswerAndOperator(df.select(element_at(col("array1"), lit(1)))) + } + } + } + } + test("CreateArray") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => @@ -2624,4 +2630,32 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { checkSparkAnswer(df.select("arrUnsupportedArgs")) } } + + test("array_contains") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = false, n = 10000) + spark.read.parquet(path.toString).createOrReplaceTempView("t1"); + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains(array(_2, _3, _4), _2) FROM t1")) + checkSparkAnswerAndOperator( + spark.sql("SELECT array_contains((CASE WHEN _2 =_3 THEN array(_4) END), _4) FROM t1")); + } + } + + test("array_remove") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled, 10000) + spark.read.parquet(path.toString).createOrReplaceTempView("t1") + checkSparkAnswerAndOperator( + sql("SELECT array_remove(array(_2, _3,_4), _2) from t1 where _2 is null")) + checkSparkAnswerAndOperator( + sql("SELECT array_remove(array(_2, _3,_4), _3) from t1 where _3 is not null")) + checkSparkAnswerAndOperator(sql( + "SELECT array_remove(case when _2 = _3 THEN array(_2, _3,_4) ELSE null END, _3) from t1")) + } + } + } } 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 e337d36c48..6795f3d55d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.optimizer.EliminateSorts import org.apache.spark.sql.comet.CometHashAggregateExec +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.{count_distinct, sum} import org.apache.spark.sql.internal.SQLConf @@ -38,6 +39,23 @@ import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + test("stddev_pop should return NaN for some cases") { + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_EXPR_STDDEV_ENABLED.key -> "true") { + Seq(true, false).foreach { nullOnDivideByZero => + withSQLConf("spark.sql.legacy.statisticalAggregate" -> nullOnDivideByZero.toString) { + + val data: Seq[(Float, Int)] = Seq((Float.PositiveInfinity, 1)) + withParquetTable(data, "tbl", false) { + val df = sql("SELECT stddev_pop(_1), stddev_pop(_2) FROM tbl") + checkSparkAnswer(df) + } + } + } + } + } + test("count with aggregation filter") { withSQLConf( CometConf.COMET_ENABLED.key -> "true", @@ -72,6 +90,37 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + // based on Spark's SQLWindowFunctionSuite test of the same name + test("window function: partition and order expressions") { + for (shuffleMode <- Seq("auto", "native", "jvm")) { + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { + val df = + Seq((1, "a", 5), (2, "a", 6), (3, "b", 7), (4, "b", 8), (5, "c", 9), (6, "c", 10)).toDF( + "month", + "area", + "product") + df.createOrReplaceTempView("windowData") + val df2 = sql(""" + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin) + checkSparkAnswer(df2) + val cometShuffles = collect(df2.queryExecution.executedPlan) { + case _: CometShuffleExchangeExec => true + } + if (shuffleMode == "jvm" || shuffleMode == "auto") { + assert(cometShuffles.length == 1) + } else { + // we fall back to Spark for shuffle because we do not support + // native shuffle with a LocalTableScan input, and we do not fall + // back to Comet columnar shuffle due to + // https://github.com/apache/datafusion-comet/issues/1248 + assert(cometShuffles.isEmpty) + } + } + } + } + test("multiple column distinct count") { withSQLConf( CometConf.COMET_ENABLED.key -> "true", @@ -925,7 +974,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("distinct") { + // TODO enable once https://github.com/apache/datafusion-comet/issues/1267 is implemented + ignore("distinct") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { Seq("native", "jvm").foreach { cometShuffleMode => withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { 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 6130e4cd58..13344c0ed5 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala @@ -1132,7 +1132,8 @@ class CometShuffleManagerSuite extends CometTestBase { partitioner = new Partitioner { override def numPartitions: Int = 50 override def getPartition(key: Any): Int = key.asInstanceOf[Int] - }) + }, + decodeTime = null) assert(CometShuffleManager.shouldBypassMergeSort(conf, dependency)) 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 2df3c74aea..2d823f804d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -22,8 +22,6 @@ package org.apache.comet.exec import java.sql.Date import java.time.{Duration, Period} -import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.util.Random import org.scalactic.source.Position @@ -68,8 +66,10 @@ class CometExecSuite extends CometTestBase { test("TopK operator should return correct results on dictionary column with nulls") { withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { withTable("test_data") { + val data = (0 to 8000) + .flatMap(_ => Seq((1, null, "A"), (2, "BBB", "B"), (3, "BBB", "B"), (4, "BBB", "B"))) val tableDF = spark.sparkContext - .parallelize(Seq((1, null, "A"), (2, "BBB", "B"), (3, "BBB", "B"), (4, "BBB", "B")), 3) + .parallelize(data, 3) .toDF("c1", "c2", "c3") tableDF .coalesce(1) @@ -462,37 +462,6 @@ class CometExecSuite extends CometTestBase { } } - test("CometExec.executeColumnarCollectIterator can collect ColumnarBatch results") { - assume(isSpark34Plus, "ChunkedByteBuffer is not serializable before Spark 3.4+") - withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "true") { - withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl") { - val df = sql("SELECT _1 + 1, _2 + 2 FROM tbl WHERE _1 > 3") - - val nativeProject = find(df.queryExecution.executedPlan) { - case _: CometProjectExec => true - case _ => false - }.get.asInstanceOf[CometProjectExec] - - val (rows, batches) = nativeProject.executeColumnarCollectIterator() - assert(rows == 46) - - val column1 = mutable.ArrayBuffer.empty[Int] - val column2 = mutable.ArrayBuffer.empty[Int] - - batches.foreach(batch => { - batch.rowIterator().asScala.foreach { row => - assert(row.numFields == 2) - column1 += row.getInt(0) - column2 += row.getInt(1) - } - }) - - assert(column1.toArray.sorted === (4 until 50).map(_ + 1).toArray) - assert(column2.toArray.sorted === (5 until 51).map(_ + 2).toArray) - } - } - } - test("scalar subquery") { val dataTypes = Seq( diff --git a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala index ad1aef4a8f..d756da1515 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -216,7 +216,7 @@ class CometJoinSuite extends CometTestBase { v.toDouble, v.toString, v % 2 == 0, - v.toString().getBytes, + v.toString.getBytes, Decimal(v)) withParquetTable((0 until 10).map(i => manyTypes(i, i % 5)), "tbl_a") { @@ -294,6 +294,7 @@ class CometJoinSuite extends CometTestBase { test("SortMergeJoin without join filter") { withSQLConf( + CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withParquetTable((0 until 10).map(i => (i, i % 5)), "tbl_a") { @@ -338,9 +339,9 @@ class CometJoinSuite extends CometTestBase { } } - // https://github.com/apache/datafusion-comet/issues/398 - ignore("SortMergeJoin with join filter") { + test("SortMergeJoin with join filter") { withSQLConf( + CometConf.COMET_EXEC_SORT_MERGE_JOIN_ENABLED.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -391,9 +392,6 @@ class CometJoinSuite extends CometTestBase { "AND tbl_a._2 >= tbl_b._1") checkSparkAnswerAndOperator(df9) - // TODO: Enable these tests after fixing the issue: - // https://github.com/apache/datafusion-comet/issues/861 - /* val df10 = sql( "SELECT * FROM tbl_a LEFT ANTI JOIN tbl_b ON tbl_a._2 = tbl_b._1 " + "AND tbl_a._2 >= tbl_b._1") @@ -403,7 +401,6 @@ class CometJoinSuite extends CometTestBase { "SELECT * FROM tbl_b LEFT ANTI JOIN tbl_a ON tbl_a._2 = tbl_b._1 " + "AND tbl_a._2 >= tbl_b._1") checkSparkAnswerAndOperator(df11) - */ } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 43f50adf60..a3906d6421 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -36,7 +36,7 @@ import org.apache.parquet.hadoop.example.ExampleParquetWriter import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark._ import org.apache.spark.internal.config.{MEMORY_OFFHEAP_ENABLED, MEMORY_OFFHEAP_SIZE, SHUFFLE_MANAGER} -import org.apache.spark.sql.comet.{CometBatchScanExec, CometBroadcastExchangeExec, CometExec, CometNativeScanExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometBroadcastExchangeExec, CometColumnarToRowExec, CometExec, CometNativeScanExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExtendedMode, InputAdapter, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -78,6 +78,7 @@ abstract class CometTestBase conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") + conf.set(CometConf.COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.key, "true") conf.set(CometConf.COMET_SPARK_TO_ARROW_ENABLED.key, "true") conf.set(CometConf.COMET_NATIVE_SCAN_ENABLED.key, "true") conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "2g") @@ -175,6 +176,7 @@ abstract class CometTestBase wrapped.foreach { case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec => case _: CometSinkPlaceHolder | _: CometScanWrapper => + case _: CometColumnarToRowExec => case _: CometSparkToColumnarExec => case _: CometExec | _: CometShuffleExchangeExec => case _: CometBroadcastExchangeExec => diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 63223ddbae..871132bd7c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -274,6 +274,7 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> dppEnabled.toString, CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", // needed for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 "spark.sql.readSideCharPadding" -> "false",