From 709d6e936bbd831043502e5d7ab74824ecba9800 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 10:35:53 -0400 Subject: [PATCH 01/35] Use Spark's RangePartitioning to compute boundary rows and serialize for native shuffle to consume. Added new test to represent #1906. --- .../scala/org/apache/comet/CometConf.scala | 2 +- docs/source/user-guide/configs.md | 2 +- native/core/benches/shuffle_writer.rs | 31 ++++++-- native/core/src/execution/planner.rs | 54 ++++++++++++- .../execution/shuffle/comet_partitioning.rs | 5 +- native/core/src/execution/shuffle/mod.rs | 2 +- .../src/execution/shuffle/shuffle_writer.rs | 77 ++++++++++--------- native/proto/src/proto/partitioning.proto | 6 +- .../shuffle/CometNativeShuffleWriter.scala | 67 +++++++++------- .../shuffle/CometShuffleDependency.scala | 4 +- .../shuffle/CometShuffleExchangeExec.scala | 60 ++++++++++++++- .../shuffle/CometShuffleManager.scala | 3 +- .../org/apache/comet/CometFuzzTestSuite.scala | 16 +++- .../comet/exec/CometNativeShuffleSuite.scala | 54 ++++++++++++- 14 files changed, 294 insertions(+), 89 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 367af5f199..8f6a833f4b 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -332,7 +332,7 @@ object CometConf extends ShimCometConf { "This feature is experimental while we investigate scenarios that don't partition data " + "correctly.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 434c1934fb..fe35d7fdbd 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -74,7 +74,7 @@ Comet provides the following configuration settings. | spark.comet.memoryOverhead | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. This config is optional. If this is not specified, it will be set to `spark.comet.memory.overhead.factor` * `spark.executor.memory`. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | | | spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 | | spark.comet.native.shuffle.partitioning.hash.enabled | Whether to enable hash partitioning for Comet native shuffle. | true | -| spark.comet.native.shuffle.partitioning.range.enabled | Experimental feature to enable range partitioning for Comet native shuffle. This feature is experimental while we investigate scenarios that don't partition data correctly. | false | +| spark.comet.native.shuffle.partitioning.range.enabled | Experimental feature to enable range partitioning for Comet native shuffle. This feature is experimental while we investigate scenarios that don't partition data correctly. | true | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | | spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false | | spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false | diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 52638d92a9..2795254ac7 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -18,6 +18,7 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; +use comet::execution::shuffle::range_partitioner::RangePartitioner; use comet::execution::shuffle::{ CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, }; @@ -84,16 +85,30 @@ fn criterion_benchmark(c: &mut Criterion) { ); } - for partitioning in [ - CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), - CometPartitioning::RangePartitioning( - LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("c0", batch.schema().as_ref()).unwrap(), - )]) - .unwrap(), + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]) + .unwrap(); + + let (owned_rows, row_converter) = { + let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( + &Vec::from(batch.columns()), + &lex_ordering, 16, + batch.num_rows(), 100, - ), + 42, + ) + .unwrap(); + ( + bounds_rows.iter().map(|row| row.owned()).collect_vec(), + row_converter, + ) + }; + + for partitioning in [ + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + CometPartitioning::RangePartitioning(lex_ordering, 16, Arc::new(row_converter), owned_rows), ] { let compression_codec = CompressionCodec::None; group.bench_function( diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0c3d345c8e..b265a0c9dc 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -85,11 +85,12 @@ use datafusion::physical_expr::LexOrdering; use crate::parquet::parquet_exec::init_datasource_exec; use arrow::array::{ - BinaryBuilder, BooleanArray, Date32Array, Decimal128Array, Float32Array, Float64Array, - Int16Array, Int32Array, Int64Array, Int8Array, NullArray, StringBuilder, + ArrayRef, BinaryBuilder, BooleanArray, Date32Array, Decimal128Array, Float32Array, + Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, NullArray, StringBuilder, TimestampMicrosecondArray, }; use arrow::buffer::BooleanBuffer; +use arrow::row::{OwnedRow, RowConverter, SortField}; use datafusion::common::utils::SingleRowListArrayBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::filter::FilterExec; @@ -2442,16 +2443,63 @@ impl PhysicalPlanner { )) } PartitioningStruct::RangePartition(range_partition) => { + // Generate the lexical ordering for comparisons let exprs: Result, ExecutionError> = range_partition .sort_orders .iter() .map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema))) .collect(); let lex_ordering = LexOrdering::new(exprs?).unwrap(); + let boundary_row_len = lex_ordering.len(); + + // Generate the row converter for comparing incoming batches to boundary rows + let sort_fields: Vec = lex_ordering + .iter() + .map(|sort_expr| { + let data_type = sort_expr.expr.data_type(input_schema.as_ref()).unwrap(); + SortField::new_with_options(data_type, sort_expr.options) + }) + .collect(); + + // Deserialize the literals to columnar collections of ScalarValues + let mut scalar_values: Vec> = vec![vec![]; lex_ordering.len()]; + range_partition + .boundary_rows + .iter() + .for_each(|boundary_row| { + assert_eq!(boundary_row.partition_bounds.len(), boundary_row_len); + // For each serialized expr in a boundary row, convert to a Literal + // expression, then extract the ScalarValue from the Literal and push it + // into the collection of ScalarValues + boundary_row.partition_bounds.iter().enumerate().for_each( + |(col_idx, literal_expr)| { + // TODO: Is there a quicker/cleaner way to go from serialized expr + // that we know is a literal to a ScalarValue? + let expr = self + .create_expr(literal_expr, Arc::clone(&input_schema)) + .unwrap(); + let literal_expr = expr.as_any().downcast_ref::().unwrap(); + scalar_values[col_idx].push(literal_expr.value().clone()); + }, + ); + }); + + // Convert the collection of ScalarValues to collection of Arrow Arrays + let arrays: Vec = scalar_values + .iter() + .map(|scalar_vec| ScalarValue::iter_to_array(scalar_vec.iter().cloned())) + .collect::, _>>()?; + + // Create a RowConverter and use to create OwnedRows from the Arrays + let converter = RowConverter::new(sort_fields)?; + let rows = converter.convert_columns(&arrays)?; + let owned_rows: Vec = rows.iter().map(|row| row.owned()).collect(); + Ok(CometPartitioning::RangePartitioning( lex_ordering, range_partition.num_partitions as usize, - range_partition.sample_size as usize, + Arc::new(converter), + owned_rows, )) } PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::SinglePartition), diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index 9c33da8e93..90f9ef9327 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow::row::{OwnedRow, RowConverter}; use datafusion::physical_expr::{LexOrdering, PhysicalExpr}; use std::sync::Arc; @@ -26,7 +27,7 @@ pub enum CometPartitioning { Hash(Vec>, usize), /// Allocate rows based on the lexical order of one of more expressions and the specified number of /// partitions - RangePartitioning(LexOrdering, usize, usize), + RangePartitioning(LexOrdering, usize, Arc, Vec), } impl CometPartitioning { @@ -34,7 +35,7 @@ impl CometPartitioning { use CometPartitioning::*; match self { SinglePartition => 1, - Hash(_, n) | RangePartitioning(_, n, _) => *n, + Hash(_, n) | RangePartitioning(_, n, _, _) => *n, } } } diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index c3cb610afa..ee770b1b85 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,7 +19,7 @@ pub(crate) mod codec; mod comet_partitioning; mod list; mod map; -mod range_partitioner; +pub mod range_partitioner; pub mod row; mod shuffle_writer; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index fe303618c3..0c6adb1f8b 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -21,7 +21,6 @@ use crate::execution::shuffle::range_partitioner::RangePartitioner; use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::interleave_record_batch; -use arrow::row::{OwnedRow, RowConverter}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; @@ -333,10 +332,6 @@ struct MultiPartitionShuffleRepartitioner { /// Reservation for repartitioning reservation: MemoryReservation, tracing_enabled: bool, - /// RangePartitioning-specific state - bounds_rows: Option>, - row_converter: Option, - seed: u64, } #[derive(Default)] @@ -413,10 +408,6 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, - bounds_rows: None, - row_converter: None, - // Spark RangePartitioner seeds off of partition number. - seed: partition as u64, }) } @@ -546,7 +537,8 @@ impl MultiPartitionShuffleRepartitioner { CometPartitioning::RangePartitioning( lex_ordering, num_output_partitions, - sample_size, + row_converter, + bounds, ) => { let mut scratch = std::mem::take(&mut self.scratch); let (partition_starts, partition_row_indices): (&Vec, &Vec) = { @@ -560,35 +552,14 @@ impl MultiPartitionShuffleRepartitioner { let num_rows = arrays[0].len(); - // If necessary (i.e., when first batch arrives) generate the bounds (as Rows) - // for range partitioning based on randomly reservoir sampling the batch. - if self.row_converter.is_none() { - let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( - &arrays, - lex_ordering, - *num_output_partitions, - input.num_rows(), - *sample_size, - self.seed, - )?; - - self.bounds_rows = - Some(bounds_rows.iter().map(|row| row.owned()).collect_vec()); - self.row_converter = Some(row_converter); - } - // Generate partition ids for every row, first by converting the partition // arrays to Rows, and then doing binary search for each Row against the // bounds Rows. - let row_batch = self - .row_converter - .as_ref() - .unwrap() - .convert_columns(arrays.as_slice())?; + let row_batch = row_converter.convert_columns(arrays.as_slice())?; RangePartitioner::partition_indices_for_batch( &row_batch, - self.bounds_rows.as_ref().unwrap().as_slice(), + bounds.as_slice(), &mut scratch.partition_ids[..num_rows], ); @@ -1278,6 +1249,7 @@ mod test { use super::*; use crate::execution::shuffle::read_ipc_compressed; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::row::{RowConverter, SortField}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; @@ -1404,15 +1376,44 @@ mod test { ) { let batch = create_batch(batch_size); + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]) + .unwrap(); + + let (owned_rows, row_converter) = if num_partitions == 1 { + let sort_fields: Vec = batch + .columns() + .iter() + .zip(&lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + (vec![], RowConverter::new(sort_fields).unwrap()) + } else { + let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( + &Vec::from(batch.columns()), + &lex_ordering, + num_partitions, + batch_size, + 100, + 42, + ) + .unwrap(); + ( + bounds_rows.iter().map(|row| row.owned()).collect_vec(), + row_converter, + ) + }; + for partitioning in [ CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), CometPartitioning::RangePartitioning( - LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", batch.schema().as_ref()).unwrap(), - )]) - .unwrap(), + lex_ordering, num_partitions, - 100, + Arc::new(row_converter), + owned_rows, ), ] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); diff --git a/native/proto/src/proto/partitioning.proto b/native/proto/src/proto/partitioning.proto index ea0e586fb1..e11d7a384b 100644 --- a/native/proto/src/proto/partitioning.proto +++ b/native/proto/src/proto/partitioning.proto @@ -42,8 +42,12 @@ message HashPartition { message SinglePartition { } +message BoundaryRow { + repeated spark.spark_expression.Expr partition_bounds = 1; +} + message RangePartition { repeated spark.spark_expression.Expr sort_orders = 1; int32 num_partitions = 2; - int32 sample_size = 3; + repeated BoundaryRow boundary_rows = 4; } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 5d772be403..58018223f7 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -28,11 +28,11 @@ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriter} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, SinglePartition} import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.comet.CometConf @@ -51,7 +51,8 @@ class CometNativeShuffleWriter[K, V]( shuffleId: Int, mapId: Long, context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter) + metricsReporter: ShuffleWriteMetricsReporter, + rangePartitionBounds: Option[Seq[InternalRow]] = None) extends ShuffleWriter[K, V] with Logging { @@ -194,35 +195,43 @@ class CometNativeShuffleWriter[K, V]( val partitioning = PartitioningOuterClass.RangePartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) - val sampleSize = { - // taken from org.apache.spark.RangePartitioner#rangeBounds - // This is the sample size we need to have roughly balanced output partitions, - // capped at 1M. - // Cast to double to avoid overflowing ints or longs - val sampleSize = math.min( - SQLConf.get - .getConf(SQLConf.RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION) - .toDouble * outputPartitioning.numPartitions, - 1e6) - // Assume the input partitions are roughly balanced and over-sample a little bit. - // Comet: we don't divide by numPartitions since each DF plan handles one partition. - math.ceil(3.0 * sampleSize).toInt - } - if (sampleSize > 8192) { - logWarning( - s"RangePartitioning sampleSize of s$sampleSize exceeds Comet RecordBatch size.") - } - partitioning.setSampleSize(sampleSize) - - val orderingExprs = rangePartitioning.ordering - .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) - if (orderingExprs.length != rangePartitioning.ordering.length) { - throw new UnsupportedOperationException( - s"Partitioning $rangePartitioning is not supported.") + { + // Serialize the ordering expressions for comparisons + val orderingExprs = rangePartitioning.ordering + .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) + if (orderingExprs.length != rangePartitioning.ordering.length) { + throw new UnsupportedOperationException( + s"Partitioning $rangePartitioning is not supported.") + } + partitioning.addAllSortOrders(orderingExprs.asJava) } - partitioning.addAllSortOrders(orderingExprs.asJava) + // Convert Spark's sequence of InternalRows that represent partitioning boundaries to sequences of Literals, + // where each outer entry represents a boundary row, and each internal entry is a value in that row. In other + // words, these are stored in row major order, not column major + val boundarySchema = rangePartitioning.ordering.flatMap(e => Some(e.dataType)) + val boundaryExprs: Seq[Seq[Literal]] = + rangePartitionBounds.get.map((row: InternalRow) => + // For every InternalRow, map its values to Literals to ao collection of Literals + row.toSeq(boundarySchema).zip(boundarySchema).map { case (value, valueType) => + Literal(value, valueType) + }) + + { + // Convert the sequences of Literals to a collection of serialized BoundaryRows + val boundaryRows: Seq[PartitioningOuterClass.BoundaryRow] = boundaryExprs + .map((rowLiterals: Seq[Literal]) => { + // Serialize each sequence of Literals as a BoundaryRow + val rowBuilder = PartitioningOuterClass.BoundaryRow.newBuilder(); + val serializedExprs = + rowLiterals.map(lit_value => + QueryPlanSerde.exprToProto(lit_value, outputAttributes).get) + rowBuilder.addAllPartitionBounds(serializedExprs.asJava) + rowBuilder.build() + }) + partitioning.addAllBoundaryRows(boundaryRows.asJava) + } val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() shuffleWriterBuilder.setPartitioning( diff --git a/spark/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 index ff35b10eb6..2b74e5a168 100644 --- a/spark/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.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.metric.SQLMetric @@ -47,7 +48,8 @@ class CometShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val outputPartitioning: Option[Partitioning] = None, val outputAttributes: Seq[Attribute] = Seq.empty, val shuffleWriteMetrics: Map[String, SQLMetric] = Map.empty, - val numParts: Int = 0) + val numParts: Int = 0, + val rangePartitionBounds: Option[Seq[InternalRow]] = None) 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 df67e7b7f2..54941b72a7 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 @@ -21,6 +21,7 @@ package org.apache.spark.sql.comet.execution.shuffle import java.util.function.Supplier +import scala.collection.JavaConverters.asScalaIteratorConverter import scala.concurrent.Future import org.apache.spark._ @@ -211,6 +212,54 @@ case class CometShuffleExchangeExec( } object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { + + /** + * Computes range partition bounds by sampling across all partitions. This ensures all executors + * use the same partition boundaries for range partitioning. + */ + def computeRangePartitionBounds( + rdd: RDD[ColumnarBatch], + rangePartitioning: RangePartitioning, + outputAttributes: Seq[Attribute]): Seq[InternalRow] = { + + // Create sampling RDD similar to existing JVM shuffle logic + val rddForSampling = rdd.mapPartitionsInternal { iter => + val projection = + UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + + // Convert ColumnarBatch to rows and project sorting columns + iter.flatMap { batch => + val rowIter = batch.rowIterator().asScala + rowIter.map { row => + // Copy the mutable keys for accurate sampling + mutablePair.update(projection(row).copy(), null) + } + } + } + + // Construct ordering on extracted sort key + val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) + + // Use Spark's RangePartitioner to compute bounds from global samples + val rangePartitioner = new RangePartitioner( + rangePartitioning.numPartitions, + rddForSampling, + ascending = true, + samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) + + // Use reflection to access the private rangeBounds field + val rangePartitionerClass = rangePartitioner.getClass + val rangeBoundsField = rangePartitionerClass.getDeclaredField("rangeBounds") + rangeBoundsField.setAccessible(true) + val rangeBounds = + rangeBoundsField.get(rangePartitioner).asInstanceOf[Array[InternalRow]].toSeq + + rangeBounds + } def prepareShuffleDependency( rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], @@ -218,6 +267,14 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer: Serializer, metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { val numParts = rdd.getNumPartitions + + // Compute range partition bounds for range partitioning + val rangePartitionBounds: Option[Seq[InternalRow]] = outputPartitioning match { + case rangePartitioning: RangePartitioning => + Some(computeRangePartitionBounds(rdd, rangePartitioning, outputAttributes)) + case _ => None + } + val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( rdd.map( (0, _) @@ -233,7 +290,8 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { outputPartitioning = Some(outputPartitioning), outputAttributes = outputAttributes, shuffleWriteMetrics = metrics, - numParts = numParts) + numParts = numParts, + rangePartitionBounds = rangePartitionBounds) dependency } 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 1142c6af17..ebc481fc85 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 @@ -209,7 +209,8 @@ class CometShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { dep.shuffleId, mapId, context, - metrics) + metrics, + dep.rangePartitionBounds) case bypassMergeSortHandle: CometBypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new CometBypassMergeSortShuffleWriter( env.blockManager, diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index ed250e141c..d6daf779ab 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -253,7 +253,20 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { df2.collect() if (usingDataSourceExec) { val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) - assert(1 == cometShuffles.length) + val expectedNumCometShuffles = CometConf.COMET_NATIVE_SCAN_IMPL.get() match { + case CometConf.SCAN_NATIVE_COMET => + // native_comet does not support reading complex types + 0 + case CometConf.SCAN_NATIVE_ICEBERG_COMPAT | CometConf.SCAN_NATIVE_DATAFUSION => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + 1 + case "native" => + // native shuffle does not support complex types as partitioning keys + 2 + } + } + assert(cometShuffles.length == expectedNumCometShuffles) } } @@ -380,6 +393,7 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach { scanImpl => super.test(testName + s" ($scanImpl, $shuffleMode shuffle)", testTags: _*) { withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl, CometConf.COMET_SCAN_ALLOW_INCOMPATIBLE.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 5e2abd7c33..624375846e 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv -import org.apache.spark.sql.{CometTestBase, DataFrame} +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col @@ -40,6 +40,7 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "native", + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { testFun } @@ -248,6 +249,57 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + withParquetTable((0 until 100000).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT * from tbl") + + // Repartition with two sort columns + val df_range_partitioned = df.repartitionByRange(10, df.col("_1"), df.col("_2")) + + val partition_bounds = df_range_partitioned.rdd + .mapPartitionsWithIndex((idx: Int, iterator: Iterator[Row]) => { + // Find the min and max value in each partition + var min: Option[Int] = None + var max: Option[Int] = None + iterator.foreach((row: Row) => { + val row_val = row.get(0).asInstanceOf[Int] + if (min.isEmpty || row_val < min.get) { + min = Some(row_val) + } + if (max.isEmpty || row_val > max.get) { + max = Some(row_val) + } + }) + Iterator.single((idx, min, max)) + }) + .collect() + + // Check min and max values in each partition + for (i <- partition_bounds.indices.init) { + val currentPartition = partition_bounds(i) + val nextPartition = partition_bounds(i + 1) + + if (currentPartition._2.isDefined && currentPartition._3.isDefined) { + val currentMin = currentPartition._2.get + val currentMax = currentPartition._3.get + assert(currentMin <= currentMax) + } + + if (currentPartition._3.isDefined && nextPartition._2.isDefined) { + val currentMax = currentPartition._3.get + val nextMin = nextPartition._2.get + assert(currentMax <= nextMin) + } + } + + } + } + + } + /** * Checks that `df` produces the same answer as Spark does, and has the `expectedNum` Comet * exchange operators. When `checkNativeOperators` is true, this also checks that all operators From 4ee3d8e214f1eb613eaa1f9ee5ae432aa850fdad Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 11:04:38 -0400 Subject: [PATCH 02/35] Fix warnings and benchmark compilation. --- native/core/benches/shuffle_writer.rs | 3 ++- native/core/src/execution/shuffle/mod.rs | 3 ++- native/core/src/execution/shuffle/range_partitioner.rs | 3 +++ .../comet/execution/shuffle/CometNativeShuffleWriter.scala | 7 ++++--- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 2795254ac7..f4d025b9f8 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -18,7 +18,7 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; -use comet::execution::shuffle::range_partitioner::RangePartitioner; +use comet::execution::shuffle::RangePartitioner; use comet::execution::shuffle::{ CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, }; @@ -32,6 +32,7 @@ use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, prelude::SessionContext, }; +use itertools::Itertools; use std::io::Cursor; use std::sync::Arc; use tokio::runtime::Runtime; diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index ee770b1b85..71869066fe 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,10 +19,11 @@ pub(crate) mod codec; mod comet_partitioning; mod list; mod map; -pub mod range_partitioner; +pub(crate) mod range_partitioner; pub mod row; mod shuffle_writer; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; pub use comet_partitioning::CometPartitioning; +pub use range_partitioner::RangePartitioner; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index e14cf61482..5dabb45635 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -31,6 +31,7 @@ impl RangePartitioner { /// We use sample_size instead of k and num_rows instead of n. /// We use indices instead of actual values in the reservoir since we'll do one take() on the /// input arrays at the end. + #[allow(dead_code)] fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { assert!(sample_size > 0); assert!( @@ -76,6 +77,7 @@ impl RangePartitioner { /// Given input arrays and range partitioning metadata: samples the input arrays, generates /// partition bounds, and returns Rows (for comparison against) and a RowConverter (for /// adapting future incoming batches). + #[allow(dead_code)] pub fn generate_bounds( partition_arrays: &Vec, lex_ordering: &LexOrdering, @@ -143,6 +145,7 @@ impl RangePartitioner { /// values since we don't have cross-partition samples to merge. /// We normalize the math to use ints instead of floating point by replacing 1.0 with a /// (imagined) num_candidates * partitions range. + #[allow(dead_code)] fn determine_bounds_for_rows( sort_fields: Vec, sampled_columns: &[ArrayRef], diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 58018223f7..290b38f880 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -207,9 +207,10 @@ class CometNativeShuffleWriter[K, V]( partitioning.addAllSortOrders(orderingExprs.asJava) } - // Convert Spark's sequence of InternalRows that represent partitioning boundaries to sequences of Literals, - // where each outer entry represents a boundary row, and each internal entry is a value in that row. In other - // words, these are stored in row major order, not column major + // Convert Spark's sequence of InternalRows that represent partitioning boundaries to + // sequences of Literals, where each outer entry represents a boundary row, and each + // internal entry is a value in that row. In other words, these are stored in row major + // order, not column major val boundarySchema = rangePartitioning.ordering.flatMap(e => Some(e.dataType)) val boundaryExprs: Seq[Seq[Literal]] = rangePartitionBounds.get.map((row: InternalRow) => From 6f34e35337d5bd5a8a24fe7e0b8cf021abd4e8c1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 11:34:51 -0400 Subject: [PATCH 03/35] Fix benchmark bug. --- native/core/benches/shuffle_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index f4d025b9f8..85da4d42d4 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -87,7 +87,7 @@ fn criterion_benchmark(c: &mut Criterion) { } let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", batch.schema().as_ref()).unwrap(), + col("c0", batch.schema().as_ref()).unwrap(), )]) .unwrap(); From 332e76afb9a158a508af0b4547a3bad3a12947b1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 11:40:51 -0400 Subject: [PATCH 04/35] Minor refactor. --- native/core/benches/shuffle_writer.rs | 25 ++++++++++--------------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 85da4d42d4..13dba9c06c 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -91,21 +91,16 @@ fn criterion_benchmark(c: &mut Criterion) { )]) .unwrap(); - let (owned_rows, row_converter) = { - let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( - &Vec::from(batch.columns()), - &lex_ordering, - 16, - batch.num_rows(), - 100, - 42, - ) - .unwrap(); - ( - bounds_rows.iter().map(|row| row.owned()).collect_vec(), - row_converter, - ) - }; + let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( + &Vec::from(batch.columns()), + &lex_ordering, + 16, + batch.num_rows(), + 100, + 42, + ) + .unwrap(); + let owned_rows = bounds_rows.iter().map(|row| row.owned()).collect_vec(); for partitioning in [ CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), From 0eb1134a20881a951bba025e4683d0e491089002 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 11:59:30 -0400 Subject: [PATCH 05/35] Cleanup to make it more clear what code came from Spark. --- .../shuffle/CometShuffleExchangeExec.scala | 59 +++++++++++-------- 1 file changed, 33 insertions(+), 26 deletions(-) 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 54941b72a7..fe576b186e 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 @@ -36,8 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.comet.{CometMetricNode, CometPlan} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeLike, ShuffleOrigin} -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec, ShuffleExchangeLike, ShuffleOrigin} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch @@ -222,34 +221,38 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { rangePartitioning: RangePartitioning, outputAttributes: Seq[Attribute]): Seq[InternalRow] = { - // Create sampling RDD similar to existing JVM shuffle logic - val rddForSampling = rdd.mapPartitionsInternal { iter => - val projection = - UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) - val mutablePair = new MutablePair[InternalRow, Null]() - - // Convert ColumnarBatch to rows and project sorting columns - iter.flatMap { batch => - val rowIter = batch.rowIterator().asScala - rowIter.map { row => - // Copy the mutable keys for accurate sampling - mutablePair.update(projection(row).copy(), null) + // The code block below is mostly brought over from + // ShuffleExchangeExec::prepareShuffleDependency and modified for columnar batches + val rangePartitioner = { + // Extract only fields used for sorting to avoid collecting large fields that does not + // affect sorting result when deciding partition bounds in RangePartitioner + val rddForSampling = rdd.mapPartitionsInternal { iter => + val projection = + UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.flatMap { batch => + val rowIter = batch.rowIterator().asScala + rowIter.map { row => + mutablePair.update(projection(row).copy(), null) + } } } - } - // Construct ordering on extracted sort key - val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => - ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + // Construct ordering on extracted sort key. + val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) + // Use Spark's RangePartitioner to compute bounds from global samples + new RangePartitioner( + rangePartitioning.numPartitions, + rddForSampling, + ascending = true, + samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) } - implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) - - // Use Spark's RangePartitioner to compute bounds from global samples - val rangePartitioner = new RangePartitioner( - rangePartitioning.numPartitions, - rddForSampling, - ascending = true, - samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) // Use reflection to access the private rangeBounds field val rangePartitionerClass = rangePartitioner.getClass @@ -260,6 +263,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { rangeBounds } + def prepareShuffleDependency( rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], @@ -284,6 +288,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { shuffleType = CometNativeShuffle, partitioner = new Partitioner { override def numPartitions: Int = outputPartitioning.numPartitions + override def getPartition(key: Any): Int = key.asInstanceOf[Int] }, decodeTime = metrics("decode_time"), @@ -381,6 +386,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { case _ => throw new IllegalStateException(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. } + def getPartitionKeyExtractor(): InternalRow => Any = newPartitioning match { case RoundRobinPartitioning(numPartitions) => // Distributes elements evenly across output partitions, starting from a random partition. @@ -433,6 +439,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { // limited range. val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer { private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix + override def computePrefix( row: InternalRow): UnsafeExternalRowSorter.PrefixComputer.Prefix = { // The hashcode generated from the binary form of a [[UnsafeRow]] should not be null. From bb67f73119f09c5e09692e1368644380126724db Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 12:11:48 -0400 Subject: [PATCH 06/35] Fix errant comment. --- spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index d6daf779ab..31d9c6cd94 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -262,7 +262,6 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { case "jvm" => 1 case "native" => - // native shuffle does not support complex types as partitioning keys 2 } } From abd895869f9360060db1082d232eb1f626e08844 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 13:50:19 -0400 Subject: [PATCH 07/35] Override partitioning scheme at serialization when num_partitions is 1. --- .../shuffle/CometNativeShuffleWriter.scala | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 290b38f880..96754e3e64 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -141,6 +141,16 @@ class CometNativeShuffleWriter[K, V]( MapStatus.apply(SparkEnv.get.blockManager.shuffleServerId, partitionLengths, mapId) } + // Spark sometimes generates partitioning schemes other than SinglePartition with + // numPartitions == 1, typically near the output of a query. In this case Comet just + // serializes a SinglePartition scheme to native. + private def isSinglePartitioning(p: Partitioning): Boolean = p match { + case SinglePartition => true + case rp: RangePartitioning => rp.numPartitions == 1 + case hp: HashPartitioning => hp.numPartitions == 1 + case _ => false + } + private def getNativePlan(dataFile: String, indexFile: String): Operator = { val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("ShuffleWriterInput") val opBuilder = OperatorOuterClass.Operator.newBuilder() @@ -171,6 +181,12 @@ class CometNativeShuffleWriter[K, V]( CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) outputPartitioning match { + case p if isSinglePartitioning(p) => + val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() + + val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() + shuffleWriterBuilder.setPartitioning( + partitioningBuilder.setSinglePartition(partitioning).build()) case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] @@ -237,12 +253,6 @@ class CometNativeShuffleWriter[K, V]( val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() shuffleWriterBuilder.setPartitioning( partitioningBuilder.setRangePartition(partitioning).build()) - case SinglePartition => - val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() - - val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() - shuffleWriterBuilder.setPartitioning( - partitioningBuilder.setSinglePartition(partitioning).build()) case _ => throw new UnsupportedOperationException( From 967d1a10e6f9ade005702ff58194faa63611048b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 28 Aug 2025 15:01:16 -0400 Subject: [PATCH 08/35] Override partitioning scheme at serialization when computed bounds result in 1 partition. --- .../shuffle/CometNativeShuffleWriter.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 96754e3e64..b11f9fead9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -141,12 +141,18 @@ class CometNativeShuffleWriter[K, V]( MapStatus.apply(SparkEnv.get.blockManager.shuffleServerId, partitionLengths, mapId) } - // Spark sometimes generates partitioning schemes other than SinglePartition with - // numPartitions == 1, typically near the output of a query. In this case Comet just - // serializes a SinglePartition scheme to native. private def isSinglePartitioning(p: Partitioning): Boolean = p match { case SinglePartition => true - case rp: RangePartitioning => rp.numPartitions == 1 + case rp: RangePartitioning => + // Spark sometimes generates RangePartitioning schemes with numPartitions == 1, + // or the computed bounds results in a single target partition. + // In this case Comet just serializes a SinglePartition scheme to native. + if ((rp.numPartitions == 1) || rangePartitionBounds.isEmpty || + rangePartitionBounds.get.isEmpty) { + true + } else { + false + } case hp: HashPartitioning => hp.numPartitions == 1 case _ => false } From 522ef802c4980975db41a8482f73ffa4c77d8594 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 29 Aug 2025 11:11:44 -0400 Subject: [PATCH 09/35] Remove string and binary range partitioning types until we sort out how to handle dictionary encoding. --- .../org/apache/comet/rules/CometExecRule.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 0dcd3828a5..63396f63b4 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -759,7 +759,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { * operations like `groupByKey`, `reduceByKey` or `join`. Native code does not support hashing * complex types, see hash_funcs/utils.rs */ - def supportedPartitioningDataType(dt: DataType): Boolean = dt match { + def supportedHashPartitioningDataType(dt: DataType): Boolean = dt match { case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | _: DecimalType | _: DateType => @@ -768,6 +768,15 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { false } + def supportedRangePartitioningDataType(dt: DataType): Boolean = dt match { + case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | + _: FloatType | _: DoubleType | _: TimestampType | _: TimestampNTZType | _: DecimalType | + _: DateType => + true + case _ => + false + } + /** * Determine which data types are supported as data columns in native shuffle. * @@ -832,7 +841,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } for (dt <- expressions.map(_.dataType).distinct) { - if (!supportedPartitioningDataType(dt)) { + if (!supportedHashPartitioningDataType(dt)) { withInfo(s, s"unsupported hash partitioning data type for native shuffle: $dt") supported = false } @@ -858,7 +867,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } for (dt <- orderings.map(_.dataType).distinct) { - if (!supportedPartitioningDataType(dt)) { + if (!supportedRangePartitioningDataType(dt)) { withInfo(s, s"unsupported range partitioning data type for native shuffle: $dt") supported = false } From 58e35b0cc435a7da4e1b40df63f20fe9eac3167c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 11 Sep 2025 09:38:48 -0700 Subject: [PATCH 10/35] Fix circular dependency in execution. --- .../shuffle/CometShuffleExchangeExec.scala | 108 ++++++++---------- 1 file changed, 48 insertions(+), 60 deletions(-) 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 fe576b186e..48a69fb373 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 @@ -212,71 +212,63 @@ case class CometShuffleExchangeExec( object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { - /** - * Computes range partition bounds by sampling across all partitions. This ensures all executors - * use the same partition boundaries for range partitioning. - */ - def computeRangePartitionBounds( + def prepareShuffleDependency( rdd: RDD[ColumnarBatch], - rangePartitioning: RangePartitioning, - outputAttributes: Seq[Attribute]): Seq[InternalRow] = { + outputAttributes: Seq[Attribute], + outputPartitioning: Partitioning, + serializer: Serializer, + metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + val numParts = rdd.getNumPartitions // The code block below is mostly brought over from - // ShuffleExchangeExec::prepareShuffleDependency and modified for columnar batches - val rangePartitioner = { - // Extract only fields used for sorting to avoid collecting large fields that does not - // affect sorting result when deciding partition bounds in RangePartitioner - val rddForSampling = rdd.mapPartitionsInternal { iter => - val projection = - UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) - val mutablePair = new MutablePair[InternalRow, Null]() - - // Internally, RangePartitioner runs a job on the RDD that samples keys to compute - // partition bounds. To get accurate samples, we need to copy the mutable keys. - iter.flatMap { batch => - val rowIter = batch.rowIterator().asScala - rowIter.map { row => - mutablePair.update(projection(row).copy(), null) + // ShuffleExchangeExec::prepareShuffleDependency + val (partitioner, rangePartitionBounds) = outputPartitioning match { + case rangePartitioning: RangePartitioning => + // Extract only fields used for sorting to avoid collecting large fields that does not + // affect sorting result when deciding partition bounds in RangePartitioner + val rddForSampling = rdd.mapPartitionsInternal { iter => + val projection = + UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.flatMap { batch => + val rowIter = batch.rowIterator().asScala + rowIter.map { row => + mutablePair.update(projection(row).copy(), null) + } } } - } - // Construct ordering on extracted sort key. - val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => - ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) - } - implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) - // Use Spark's RangePartitioner to compute bounds from global samples - new RangePartitioner( - rangePartitioning.numPartitions, - rddForSampling, - ascending = true, - samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) - } + // Construct ordering on extracted sort key. + val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) + // Use Spark's RangePartitioner to compute bounds from global samples + val rangePartitioner = new RangePartitioner( + rangePartitioning.numPartitions, + rddForSampling, + ascending = true, + samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) - // Use reflection to access the private rangeBounds field - val rangePartitionerClass = rangePartitioner.getClass - val rangeBoundsField = rangePartitionerClass.getDeclaredField("rangeBounds") - rangeBoundsField.setAccessible(true) - val rangeBounds = - rangeBoundsField.get(rangePartitioner).asInstanceOf[Array[InternalRow]].toSeq + // Use reflection to access the private rangeBounds field + val rangeBoundsField = rangePartitioner.getClass.getDeclaredField("rangeBounds") + rangeBoundsField.setAccessible(true) + val rangeBounds = + rangeBoundsField.get(rangePartitioner).asInstanceOf[Array[InternalRow]].toSeq - rangeBounds - } + (rangePartitioner.asInstanceOf[Partitioner], Some(rangeBounds)) - def prepareShuffleDependency( - rdd: RDD[ColumnarBatch], - outputAttributes: Seq[Attribute], - outputPartitioning: Partitioning, - serializer: Serializer, - metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { - val numParts = rdd.getNumPartitions + case _ => + ( + new Partitioner { + override def numPartitions: Int = outputPartitioning.numPartitions - // Compute range partition bounds for range partitioning - val rangePartitionBounds: Option[Seq[InternalRow]] = outputPartitioning match { - case rangePartitioning: RangePartitioning => - Some(computeRangePartitionBounds(rdd, rangePartitioning, outputAttributes)) - case _ => None + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + }, + None) } val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( @@ -286,11 +278,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer = serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(metrics), shuffleType = CometNativeShuffle, - partitioner = new Partitioner { - override def numPartitions: Int = outputPartitioning.numPartitions - - override def getPartition(key: Any): Int = key.asInstanceOf[Int] - }, + partitioner = partitioner, decodeTime = metrics("decode_time"), outputPartitioning = Some(outputPartitioning), outputAttributes = outputAttributes, From 08a4b519368fb5e1f8e0d722385177423ff3f690 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 11 Sep 2025 10:27:26 -0700 Subject: [PATCH 11/35] Update plans. --- .../approved-plans-v1_4-spark3_5/q2/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q2/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q39a/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q39a/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q39b/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q39b/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q71/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q71/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q73/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q73/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q91/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q91/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q39a/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q39a/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q39b/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q39b/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q73/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q73/simplified.txt | 2 +- .../tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt | 6 +++--- .../approved-plans-v1_4/q2/simplified.txt | 2 +- .../approved-plans-v1_4/q39a/explain.txt | 6 +++--- .../approved-plans-v1_4/q39a/simplified.txt | 2 +- .../approved-plans-v1_4/q39b/explain.txt | 6 +++--- .../approved-plans-v1_4/q39b/simplified.txt | 2 +- .../approved-plans-v1_4/q71/explain.txt | 6 +++--- .../approved-plans-v1_4/q71/simplified.txt | 2 +- .../approved-plans-v1_4/q73/explain.txt | 6 +++--- .../approved-plans-v1_4/q73/simplified.txt | 2 +- .../approved-plans-v1_4/q91/explain.txt | 6 +++--- .../approved-plans-v1_4/q91/simplified.txt | 2 +- 30 files changed, 60 insertions(+), 60 deletions(-) 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 213f1c61b4..58e7d31f47 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] Arguments: [d_week_seq1#29, 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#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, 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 25ed325468..420fd43d85 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [d_week_seq1] #1 CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] 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 8e6d9265d6..ed74780c2c 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] 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 adb788d374..4a5376e4f7 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 69b6185645..2a31b0d8b2 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] 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 adb788d374..4a5376e4f7 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 9ecf9be21f..bb217d6417 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] 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 a791eee1ba..a083a4398d 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] 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 2de3317bc8..346e110371 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 5d798743dd..c8f7f27fba 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] 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 0ba113dd4d..cf532d2b41 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] 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 5be6489e2d..a1b8084d84 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] 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 e5b1fc495c..4e7b77d7ec 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange 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#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] 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 0fccdf5d59..cd8f2c926b 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 80ef707d35..7d8df40ba0 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange 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#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] 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 0fccdf5d59..cd8f2c926b 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 87b805556e..3bb13dcb30 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 5d798743dd..c8f7f27fba 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] 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 00c1602acc..e13ae7371e 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] 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 25ed325468..420fd43d85 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [d_week_seq1] #1 CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] 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 8e6d9265d6..ed74780c2c 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] 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 adb788d374..4a5376e4f7 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 69b6185645..2a31b0d8b2 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] 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 adb788d374..4a5376e4f7 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] 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 9ecf9be21f..bb217d6417 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] 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 a791eee1ba..a083a4398d 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] 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 2de3317bc8..346e110371 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 5d798743dd..c8f7f27fba 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] 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 0ba113dd4d..cf532d2b41 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] 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 5be6489e2d..a1b8084d84 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] From 2f4280f93c5a116c06bf83a33137c0da7e96688f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 11 Sep 2025 14:18:43 -0700 Subject: [PATCH 12/35] fix Spark SQL test "change SQLConf should not change view behavior - orderByOrdinal"? --- native/core/src/execution/planner.rs | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 976dc4904f..a1e50972a9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2455,7 +2455,6 @@ impl PhysicalPlanner { .map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema))) .collect(); let lex_ordering = LexOrdering::new(exprs?).unwrap(); - let boundary_row_len = lex_ordering.len(); // Generate the row converter for comparing incoming batches to boundary rows let sort_fields: Vec = lex_ordering @@ -2472,21 +2471,28 @@ impl PhysicalPlanner { .boundary_rows .iter() .for_each(|boundary_row| { - assert_eq!(boundary_row.partition_bounds.len(), boundary_row_len); // For each serialized expr in a boundary row, convert to a Literal // expression, then extract the ScalarValue from the Literal and push it // into the collection of ScalarValues - boundary_row.partition_bounds.iter().enumerate().for_each( - |(col_idx, literal_expr)| { - // TODO: Is there a quicker/cleaner way to go from serialized expr - // that we know is a literal to a ScalarValue? + + lex_ordering + .iter() + .enumerate() + .for_each(|(col_idx, sort_expr)| { let expr = self - .create_expr(literal_expr, Arc::clone(&input_schema)) + .create_expr( + &boundary_row.partition_bounds[sort_expr + .expr + .as_any() + .downcast_ref::() + .unwrap() + .index()], + Arc::clone(&input_schema), + ) .unwrap(); let literal_expr = expr.as_any().downcast_ref::().unwrap(); scalar_values[col_idx].push(literal_expr.value().clone()); - }, - ); + }); }); // Convert the collection of ScalarValues to collection of Arrow Arrays From 58f2edaa0ff8beb6c1ba5b36b88779695c9b05af Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 11 Sep 2025 21:38:50 -0700 Subject: [PATCH 13/35] Fix bug with indexing into boundary rows. --- native/core/src/execution/planner.rs | 29 +++++++------------ .../org/apache/comet/CometFuzzTestSuite.scala | 15 ++++++++++ 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index a1e50972a9..1d2c720541 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2474,25 +2474,16 @@ impl PhysicalPlanner { // For each serialized expr in a boundary row, convert to a Literal // expression, then extract the ScalarValue from the Literal and push it // into the collection of ScalarValues - - lex_ordering - .iter() - .enumerate() - .for_each(|(col_idx, sort_expr)| { - let expr = self - .create_expr( - &boundary_row.partition_bounds[sort_expr - .expr - .as_any() - .downcast_ref::() - .unwrap() - .index()], - Arc::clone(&input_schema), - ) - .unwrap(); - let literal_expr = expr.as_any().downcast_ref::().unwrap(); - scalar_values[col_idx].push(literal_expr.value().clone()); - }); + (0..lex_ordering.len()).for_each(|col_idx| { + let expr = self + .create_expr( + &boundary_row.partition_bounds[col_idx], + Arc::clone(&input_schema), + ) + .unwrap(); + let literal_expr = expr.as_any().downcast_ref::().unwrap(); + scalar_values[col_idx].push(literal_expr.value().clone()); + }); }); // Convert the collection of ScalarValues to collection of Arrow Arrays diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index 31d9c6cd94..978fbbfab4 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -192,6 +192,21 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("order by random columns") { + val df = spark.read.parquet(filename) + df.createOrReplaceTempView("t1") + + for (_ <- 1 to 10) { + // We only do order by primitive, non-variable length types to exercise native shuffle's + // RangePartitioning which only supports those types. + val shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 12).toList) + val randomSize = Random.nextInt(shuffledPrimitiveCols.length) + 1 + val randomColsSubset = shuffledPrimitiveCols.take(randomSize).toArray.mkString(",") + val sql = s"SELECT $randomColsSubset FROM t1 ORDER BY $randomColsSubset" + checkSparkAnswerAndOperator(sql) + } + } + test("aggregate group by single column") { val df = spark.read.parquet(filename) df.createOrReplaceTempView("t1") From 044e098b9f0ea1c0839af7c617afc3dbf6315626 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 16 Sep 2025 16:48:20 -0400 Subject: [PATCH 14/35] Remove range_partitioner.rs (native bounds calculation and reservoir sampling). --- native/core/benches/shuffle_writer.rs | 34 +- native/core/src/execution/shuffle/mod.rs | 2 - .../execution/shuffle/range_partitioner.rs | 477 ------------------ .../src/execution/shuffle/shuffle_writer.rs | 70 +-- 4 files changed, 63 insertions(+), 520 deletions(-) delete mode 100644 native/core/src/execution/shuffle/range_partitioner.rs diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 13dba9c06c..70f1acc8db 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -16,9 +16,9 @@ // under the License. use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; -use arrow::array::{builder::StringBuilder, RecordBatch}; +use arrow::array::{builder::StringBuilder, Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; -use comet::execution::shuffle::RangePartitioner; +use arrow::row::{RowConverter, SortField}; use comet::execution::shuffle::{ CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, }; @@ -91,15 +91,27 @@ fn criterion_benchmark(c: &mut Criterion) { )]) .unwrap(); - let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( - &Vec::from(batch.columns()), - &lex_ordering, - 16, - batch.num_rows(), - 100, - 42, - ) - .unwrap(); + let sort_fields: Vec = batch + .columns() + .iter() + .zip(&lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + let row_converter = RowConverter::new(sort_fields).unwrap(); + + // These are hard-coded values based on the benchmark params of 8192 rows per batch, and 16 + // partitions. If these change, these values need to be recalculated, or bring over the + // bounds-finding logic from shuffle_write_test in shuffle_writer.rs. + let bounds_ints = vec![ + 512, 1024, 1536, 2048, 2560, 3072, 3584, 4096, 4608, 5120, 5632, 6144, 6656, 7168, 7680, + ]; + let bounds_array: Arc = Arc::new(Int32Array::from(bounds_ints)); + let bounds_rows = row_converter + .convert_columns(vec![bounds_array].as_slice()) + .unwrap(); + let owned_rows = bounds_rows.iter().map(|row| row.owned()).collect_vec(); for partitioning in [ diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index 71869066fe..e2798df63e 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,11 +19,9 @@ pub(crate) mod codec; mod comet_partitioning; mod list; mod map; -pub(crate) mod range_partitioner; pub mod row; mod shuffle_writer; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; pub use comet_partitioning::CometPartitioning; -pub use range_partitioner::RangePartitioner; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs deleted file mode 100644 index 5dabb45635..0000000000 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ /dev/null @@ -1,477 +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 arrow::array::{ArrayRef, UInt64Array}; -use arrow::compute::{take_arrays, TakeOptions}; -use arrow::row::{OwnedRow, Row, RowConverter, Rows, SortField}; -use datafusion::error::{DataFusionError, Result}; -use datafusion::physical_expr::LexOrdering; -use rand::{rngs::SmallRng, Rng, SeedableRng}; - -pub struct RangePartitioner; - -impl RangePartitioner { - /// Given a number of rows, sample size, and a random seed, generates unique indices to take - /// from an input batch to act as a random sample. - /// Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L - /// We use sample_size instead of k and num_rows instead of n. - /// We use indices instead of actual values in the reservoir since we'll do one take() on the - /// input arrays at the end. - #[allow(dead_code)] - fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { - assert!(sample_size > 0); - assert!( - num_rows > sample_size, - "Sample size > num_rows yields original batch." - ); - - // Initialize our reservoir with indices of the first |sample_size| elements. - let mut reservoir: Vec = (0..sample_size as u64).collect(); - - let mut rng = SmallRng::seed_from_u64(seed); - let mut w = (rng.random::().ln() / sample_size as f64).exp(); - let mut i = sample_size - 1; - - while i < num_rows { - i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize + 1; - - if i < num_rows { - // Replace a random item in the reservoir with i - let random_index = rng.random_range(0..sample_size); - reservoir[random_index] = i as u64; - w *= (rng.random::().ln() / sample_size as f64).exp(); - } - } - - reservoir - } - - /// Given a batch of Rows, an ordered vector of Rows that represent partition boundaries, and - /// a slice with enough space for the input batch, determines a partition id for every input - /// Row using binary search. - pub fn partition_indices_for_batch( - row_batch: &Rows, - partition_bounds_vec: &[OwnedRow], - partition_ids: &mut [u32], - ) { - row_batch.iter().enumerate().for_each(|(row_idx, row)| { - partition_ids[row_idx] = - partition_bounds_vec.partition_point(|bound| bound.row() <= row) as u32 - }); - } - - /// Given input arrays and range partitioning metadata: samples the input arrays, generates - /// partition bounds, and returns Rows (for comparison against) and a RowConverter (for - /// adapting future incoming batches). - #[allow(dead_code)] - pub fn generate_bounds( - partition_arrays: &Vec, - lex_ordering: &LexOrdering, - num_output_partitions: usize, - num_rows: usize, - sample_size: usize, - seed: u64, - ) -> Result<(Rows, RowConverter), DataFusionError> { - let sampled_columns = if sample_size < num_rows { - // Construct our sample indices. - let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( - num_rows, - sample_size, - seed, - )); - - // Extract our sampled data from the input data. - take_arrays( - partition_arrays, - &sample_indices, - Some(TakeOptions { - check_bounds: false, - }), - )? - } else { - // Requested sample_size is larger than the batch, so just use the batch. - partition_arrays.clone() - }; - - // Generate our bounds indices. - let sort_fields: Vec = partition_arrays - .iter() - .zip(lex_ordering) - .map(|(array, sort_expr)| { - SortField::new_with_options(array.data_type().clone(), sort_expr.options) - }) - .collect(); - - let (bounds_indices, row_converter) = RangePartitioner::determine_bounds_for_rows( - sort_fields, - sampled_columns.as_slice(), - num_output_partitions, - )?; - - // Extract our bounds data from the sampled data. - let bounds_indices_array = UInt64Array::from(bounds_indices); - let bounds_arrays = take_arrays( - sampled_columns.as_slice(), - &bounds_indices_array, - Some(TakeOptions { - check_bounds: false, - }), - )?; - - // Convert the bounds data to Rows and return with RowConverter. - Ok(( - row_converter.convert_columns(bounds_arrays.as_slice())?, - row_converter, - )) - } - - /// Given a sort ordering, sampled data, and a number of target partitions, finds the partition - /// bounds and returns them as indices into the sampled data. - /// Adapted from org.apache.spark.RangePartitioner.determineBounds but without weighted - /// values since we don't have cross-partition samples to merge. - /// We normalize the math to use ints instead of floating point by replacing 1.0 with a - /// (imagined) num_candidates * partitions range. - #[allow(dead_code)] - fn determine_bounds_for_rows( - sort_fields: Vec, - sampled_columns: &[ArrayRef], - partitions: usize, - ) -> Result<(Vec, RowConverter), DataFusionError> { - assert!(partitions > 1); - - let converter = RowConverter::new(sort_fields)?; - let sampled_rows = converter.convert_columns(sampled_columns)?; - let mut sorted_sampled_rows: Vec<(usize, Row)> = sampled_rows.iter().enumerate().collect(); - sorted_sampled_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - - let num_candidates = sampled_rows.num_rows(); - let step = 1.0 / partitions as f64; - let mut cumulative_weights = 0.0; - let mut target = step; - let mut bounds_indices: Vec = Vec::with_capacity(partitions - 1); - let mut i = 0; - let mut j = 0; - let mut previous_bound = None; - let sample_weight = 1.0 / num_candidates as f64; - while (i < num_candidates) && (j < partitions - 1) { - let key = sorted_sampled_rows[i]; - cumulative_weights += sample_weight; - if cumulative_weights >= target { - // Skip duplicate values. - if previous_bound.is_none() || key.1 > previous_bound.unwrap() { - bounds_indices.push(key.0 as u64); - target += step; - j += 1; - previous_bound = Some(key.1) - } - } - i += 1 - } - - Ok((bounds_indices, converter)) - } -} - -#[cfg(test)] -mod test { - use super::*; - use arrow::array::{Array, AsArray, Int64Array, RecordBatch, UInt64Array}; - use arrow::compute::take_record_batch; - use arrow::datatypes::DataType::{Float64, Int64}; - use arrow::datatypes::{Field, Float64Type, Int32Type, Int64Type, Schema}; - use datafusion::common::record_batch; - use datafusion::physical_expr::expressions::col; - use datafusion::physical_expr::PhysicalSortExpr; - use itertools::Itertools; - use std::sync::Arc; - - fn sample_batch(input: RecordBatch, indices: Vec) -> RecordBatch { - let indices = UInt64Array::from(indices); - take_record_batch(&input, &indices).unwrap() - } - - fn check_sample_indices(indices: &[u64], batch_size: usize, sample_size: usize) { - // sample indices size should never exceed the batch size - assert!(indices.len() <= batch_size); - // number of samples should be the smaller of batch size and sample size - assert_eq!(indices.len(), batch_size.min(sample_size)); - // Check that indices are not out of bounds - indices - .iter() - .for_each(|&idx| assert!(idx < batch_size as u64)); - // Check that values are distinct - let sorted_indices = indices.iter().sorted().collect_vec(); - assert_eq!( - sorted_indices.len(), - sorted_indices.iter().dedup().collect_vec().len() - ); - } - - fn check_bounds_indices(indices: &[u64], sample_size: usize) { - // bounds indices size should never exceed the sample size - assert!(indices.len() <= sample_size); - // Check that indices are not out of bounds - indices - .iter() - .for_each(|&idx| assert!(idx < sample_size as u64)); - // Check that values are distinct - let sorted_indices = indices.iter().sorted().collect_vec(); - assert_eq!( - sorted_indices.len(), - sorted_indices.iter().dedup().collect_vec().len() - ); - } - - #[test] - // Reservoir sampling from a sorted batch can appear like an adversarial case, so we want to - // ensure that we still generate reasonable bounds. A sorted batch with partitioning column of - // [0,8192) is sampled and we generate bounds for 10 partitions. A perfect split would have - // ~819 between each partition, so we just assert that the difference in bounds is <1000. - fn generate_bounds_for_sorted_batch() { - let input_batch = create_random_batch(8192, true, Some((0, 8192))); - - let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", input_batch.schema().as_ref()).unwrap(), - )]); - - let (rows, row_converter) = RangePartitioner::generate_bounds( - input_batch.columns().to_vec().as_ref(), - &lex_ordering.unwrap(), - 10, - input_batch.num_rows(), - 1000, - 42, - ) - .unwrap(); - - let rows_array = row_converter.convert_rows(&rows).unwrap(); - - let primitive_array = rows_array[0].as_primitive::(); - - for i in 0..8 { - assert!( - (primitive_array.values().get(i + 1).unwrap() - - primitive_array.values().get(i).unwrap()) - .abs() - < 1000 - ); - } - } - - #[test] - // We want to verify that with hand-written bounds for a distribution of data that we get - // reasonable partition indices for a random batch. For this scenario, we create a full - // RecordBatch with one partition column. The values in the column are uniform randomly - // distributed between [0,10). We request 10 partitions with bounds of [1,2,3,4,5,6,7,8,9], - // and the result should be 10 bins with reasonably close counts. - fn partition_indices_for_batch() { - let sort_fields = vec![SortField::new(Int64)]; - let row_converter = RowConverter::new(sort_fields).unwrap(); - let mut partition_ids = vec![0u32; 8192]; - let mut partition_counts = [0u32; 10]; - - let input_batch = create_random_batch(8192, false, Some((0, 10))); - let bounds = record_batch!(("a", Int64, (1..=9).collect_vec())).unwrap(); - - let input_rows = row_converter - .convert_columns(input_batch.columns()) - .unwrap(); - - let bounds_rows = row_converter.convert_columns(bounds.columns()).unwrap(); - - let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); - - RangePartitioner::partition_indices_for_batch( - &input_rows, - owned_rows_vec.as_slice(), - &mut partition_ids, - ); - - partition_ids - .iter() - .for_each(|&partition_id| partition_counts[partition_id as usize] += 1); - - // The RecordBatch won't be perfectly distributed, so for 8192 / 10 possible values we - // check that each partition just has >700 values in them. - partition_counts - .iter() - .for_each(|&partition_count| assert!(partition_count > 700)); - } - - #[test] - // We want to verify that reservoir sampling yields valid indices for different size input - // batches. We randomly generate batch sizes and sample sizes, and then construct reservoir - // samples for each scenario. Finally, we validate the indices. - fn reservoir_sample_random() { - let mut rng = SmallRng::seed_from_u64(42); - - // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI - // when testing with Miri. - for _ in 0..if cfg!(miri) { 8 } else { 8192 } { - let batch_size: usize = rng.random_range(1..=8192); - // We don't test sample size > batch_size since in that case you would just take the - // entire batch as the sample. - let sample_size: usize = rng.random_range(1..batch_size); - let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size, 42); - - check_sample_indices(&indices, batch_size, sample_size); - } - } - - #[test] - // org.apache.spark.util.random.SamplingUtilsSuite - // "SPARK-18678 reservoirSampleAndCount with tiny input" - fn reservoir_sample_and_count_with_tiny_input() { - let batch = record_batch!(("a", Int32, vec![0, 1])).unwrap(); - let mut counts: Vec = vec![0; 2]; - for i in 0..500 { - let indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 1, i); - let result = sample_batch(batch.clone(), indices); - assert_eq!(result.num_rows(), 1); - counts[result.column(0).as_primitive::().value(0) as usize] += 1; - } - // If correct, should be true with prob ~ 0.99999707 according to original Spark test. - assert!((counts[0] - counts[1]).abs() <= 100) - } - - #[test] - // org.apache.spark.PartitioningSuite - // "RangePartitioner.determineBounds" - fn determine_bounds_for_rows() { - // The original test had weights on the values. We just duplicate them because our - // determine_bounds function is unweighted. - let batch = record_batch!(( - "a", - Float64, - vec![ - Some(0.7), - Some(0.7), - Some(0.1), - Some(0.4), - Some(0.3), - Some(0.2), - Some(0.5), - Some(1.0), - Some(1.0), - Some(1.0), - ] - )) - .unwrap(); - - let sort_fields = vec![SortField::new(Float64)]; - - let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3).unwrap(); - - check_bounds_indices(rows.as_slice(), batch.num_rows()); - - assert_eq!(rows.len(), 2); - - let indices = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &indices).unwrap(); - let bounds_array = bounds.column(0).as_primitive::(); - assert_eq!(bounds_array.values(), &[0.4, 0.7]); - } - - #[test] - // We want to verify that determining bounds yields valid indices for different size sample - // batches. We randomly generate batches and number of partitions, and then construct - // bounds for each scenario. Finally, we validate the indices. - fn determine_bounds_random() { - let mut rng = SmallRng::seed_from_u64(42); - - let sort_fields = vec![SortField::new(Int64)]; - - // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI - // when testing with Miri. - for _ in 0..if cfg!(miri) { 8 } else { 2048 } { - let batch_size = rng.random_range(0..=8192); - // We don't test fewer than 2 partitions since this is used by the - // MultiPartitionShuffleRepartitioner which is for >1 partitions. - let num_partitions = rng.random_range(2..=10000); - - let batch = create_random_batch(batch_size, false, None); - - let (rows, _) = RangePartitioner::determine_bounds_for_rows( - sort_fields.clone(), - batch.columns(), - num_partitions, - ) - .unwrap(); - - check_bounds_indices(rows.as_slice(), batch_size as usize); - - let rows_array = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &rows_array).unwrap(); - - let bounds_vec: Vec = bounds - .column(0) - .as_primitive::() - .values() - .to_vec(); - - // Bounds should be sorted. - assert!(bounds_vec.is_sorted()); - // Bounds should be unique. - assert_eq!( - bounds_vec.len(), - bounds_vec.iter().dedup().collect_vec().len() - ); - } - } - - #[test] - // We want to make sure that finding bounds works with nulls. DF has more exhaustive tests for - // sorting with nulls, so we defer to those for more coverage. This is just a small - // deterministic test to verify that nulls can be partition boundaries. - fn determine_bounds_with_nulls() { - let batch = record_batch!(("a", Float64, vec![None, None, Some(0.1),])).unwrap(); - - let sort_fields = vec![SortField::new(Float64)]; - - let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2).unwrap(); - - assert_eq!(rows.len(), 1); - - let indices = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &indices).unwrap(); - let bounds_array = bounds.column(0).as_primitive::(); - assert!(bounds_array.is_null(0)); - } - - fn create_random_batch(batch_size: u32, sort: bool, range: Option<(i64, i64)>) -> RecordBatch { - let mut rng = SmallRng::seed_from_u64(42); - let mut column: Vec = if let Some((min, max)) = range { - assert!(min <= max); - (0..batch_size) - .map(|_| rng.random_range(min..max)) - .collect() - } else { - (0..batch_size).map(|_| rng.random::()).collect() - }; - if sort { - column.sort(); - } - let array = Int64Array::from(column); - let schema = Arc::new(Schema::new(vec![Field::new("a", Int64, true)])); - RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() - } -} diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 0c6adb1f8b..53c48f2afc 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,7 +17,6 @@ //! Defines the External shuffle repartition plan. -use crate::execution::shuffle::range_partitioner::RangePartitioner; use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::interleave_record_batch; @@ -555,13 +554,17 @@ impl MultiPartitionShuffleRepartitioner { // Generate partition ids for every row, first by converting the partition // arrays to Rows, and then doing binary search for each Row against the // bounds Rows. - let row_batch = row_converter.convert_columns(arrays.as_slice())?; + { + let row_batch = row_converter.convert_columns(arrays.as_slice())?; + let partition_ids = &mut scratch.partition_ids[..num_rows]; - RangePartitioner::partition_indices_for_batch( - &row_batch, - bounds.as_slice(), - &mut scratch.partition_ids[..num_rows], - ); + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + partition_ids[row_idx] = bounds + .as_slice() + .partition_point(|bound| bound.row() <= row) + as u32 + }); + } // We now have partition ids for every input row, map that to partition starts // and partition indices to eventually right these rows to partition buffers. @@ -1381,30 +1384,37 @@ mod test { )]) .unwrap(); - let (owned_rows, row_converter) = if num_partitions == 1 { - let sort_fields: Vec = batch - .columns() - .iter() - .zip(&lex_ordering) - .map(|(array, sort_expr)| { - SortField::new_with_options(array.data_type().clone(), sort_expr.options) - }) - .collect(); - (vec![], RowConverter::new(sort_fields).unwrap()) + let sort_fields: Vec = batch + .columns() + .iter() + .zip(&lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + let row_converter = RowConverter::new(sort_fields).unwrap(); + + let owned_rows = if num_partitions == 1 { + vec![] } else { - let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( - &Vec::from(batch.columns()), - &lex_ordering, - num_partitions, - batch_size, - 100, - 42, - ) - .unwrap(); - ( - bounds_rows.iter().map(|row| row.owned()).collect_vec(), - row_converter, - ) + // Determine range boundaries based on create_batch implementation. We just divide the + // domain of values in the batch equally to find partition bounds. + let bounds_strings = { + let mut boundaries = Vec::with_capacity(num_partitions - 1); + let step = batch_size as f64 / num_partitions as f64; + + for i in 1..(num_partitions) { + boundaries.push(Some((step * i as f64).round().to_string())); + } + boundaries + }; + let bounds_array: Arc = Arc::new(StringArray::from(bounds_strings)); + let bounds_rows = row_converter + .convert_columns(vec![bounds_array].as_slice()) + .unwrap(); + + let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); + owned_rows_vec }; for partitioning in [ From 21c4665bd16f5b728d5ff860e0faec720380c305 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 16 Sep 2025 16:54:09 -0400 Subject: [PATCH 15/35] remove errant collection.JavaConverters --- .../sql/comet/execution/shuffle/CometShuffleExchangeExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 48a69fb373..1f7d37a108 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 @@ -21,8 +21,8 @@ package org.apache.spark.sql.comet.execution.shuffle import java.util.function.Supplier -import scala.collection.JavaConverters.asScalaIteratorConverter import scala.concurrent.Future +import scala.jdk.CollectionConverters._ import org.apache.spark._ import org.apache.spark.internal.config From c9acdfc911a004712856bd6d70abe250f300974c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 16 Sep 2025 17:21:14 -0400 Subject: [PATCH 16/35] Remove redundant config (setting to default). --- .../scala/org/apache/comet/exec/CometNativeShuffleSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 624375846e..5d49fe12fb 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -40,7 +40,6 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_MODE.key -> "native", - CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { testFun } From 261ea3307b90c5b76f013886915cf8181e9b7103 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 17 Sep 2025 15:23:27 -0400 Subject: [PATCH 17/35] Hoist wrapping the child of ShuffleWriter in a CopyExec before we generate the partitioning scheme. This solves the issue where the input schema says it contains dictionaries that were later going to be unpacked by CopyExec. Will open an issue to understand why we even wrap the child in CopyExec in the first place. --- native/core/src/execution/planner.rs | 17 +++++++++++------ .../org/apache/comet/rules/CometExecRule.scala | 4 ++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 6eb807a19c..8b45f1efc0 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -484,7 +484,7 @@ impl PhysicalPlanner { ))) } } - }, + } Value::ListVal(values) => { if let DataType::List(f) = data_type { match f.data_type() { @@ -597,11 +597,10 @@ impl PhysicalPlanner { ))) } } - } else { return Err(GeneralError(format!( "Expected DataType::List but got {data_type:?}" - ))) + ))); } } } @@ -1510,8 +1509,14 @@ impl PhysicalPlanner { assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; - let partitioning = self - .create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?; + // We wrap native shuffle in a CopyExec. This existed previously, but for + // RangePartitioning at least we want to ensure that dictionaries are unpacked. + let wrapped_child = Self::wrap_in_copy_exec(Arc::clone(&child.native_plan)); + + let partitioning = self.create_partitioning( + writer.partitioning.as_ref().unwrap(), + wrapped_child.schema(), + )?; let codec = match writer.codec.try_into() { Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), @@ -1527,7 +1532,7 @@ impl PhysicalPlanner { }?; let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( - Self::wrap_in_copy_exec(Arc::clone(&child.native_plan)), + wrapped_child, partitioning, codec, writer.output_data_file.clone(), diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index c32717504c..30dfdabbf6 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -781,8 +781,8 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { def supportedRangePartitioningDataType(dt: DataType): Boolean = dt match { case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | - _: FloatType | _: DoubleType | _: TimestampType | _: TimestampNTZType | _: DecimalType | - _: DateType => + _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | + _: TimestampNTZType | _: DecimalType | _: DateType => true case _ => false From 4077f7d89d639af90f5625a554f07062cacc28d7 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 17 Sep 2025 15:26:37 -0400 Subject: [PATCH 18/35] Update test after last commit. --- .../src/test/scala/org/apache/comet/CometFuzzTestSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index 4efb19d904..398d960136 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -138,9 +138,9 @@ class CometFuzzTestSuite extends CometFuzzTestBase { df.createOrReplaceTempView("t1") for (_ <- 1 to 10) { - // We only do order by primitive, non-variable length types to exercise native shuffle's + // We only do order by permutations of primitive types to exercise native shuffle's // RangePartitioning which only supports those types. - val shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 12).toList) + val shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 14).toList) val randomSize = Random.nextInt(shuffledPrimitiveCols.length) + 1 val randomColsSubset = shuffledPrimitiveCols.take(randomSize).toArray.mkString(",") val sql = s"SELECT $randomColsSubset FROM t1 ORDER BY $randomColsSubset" From 764675b20982a1d6b10fff506b11f88abb63ad54 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 19 Sep 2025 17:02:41 -0400 Subject: [PATCH 19/35] Add a reduced test case for the Spark SQL test failure. --- .../comet/exec/CometNativeShuffleSuite.scala | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 5d49fe12fb..1318f7ca5b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -35,7 +35,7 @@ import org.apache.comet.CometConf class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { + pos: Position): Unit = { super.test(testName, testTags: _*) { withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", @@ -248,6 +248,21 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } + // This duplicates behavior seen in a much more complicated Spark SQL test + // "SPARK-44647: test join key is the second cluster key" + test("range partitioning with duplicate column references") { + withParquetTable((0 until 100).map(i => (i % 10, (i % 5).toByte, i % 3)), "tbl") { + + val df = sql("SELECT * FROM tbl") + + // This is the problematic case: duplicate references to the same column + // Both _1 references should get different BoundReference indices + val rangePartitioned = df.repartitionByRange(3, df.col("_1"), df.col("_1"), df.col("_2")) + + checkShuffleAnswer(rangePartitioned, 1) + } + } + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to // test for incorrect partition values after native RangePartitioning test("fix: range partitioning #1906") { @@ -305,9 +320,9 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper * used by `df` are Comet native operators. */ private def checkShuffleAnswer( - df: DataFrame, - expectedNum: Int, - checkNativeOperators: Boolean = false): Unit = { + df: DataFrame, + expectedNum: Int, + checkNativeOperators: Boolean = false): Unit = { checkCometExchange(df, expectedNum, true) if (checkNativeOperators) { checkSparkAnswerAndOperator(df) From 91fef942e799157c3d0346e0055e44472d529cfa Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 19 Sep 2025 17:34:55 -0400 Subject: [PATCH 20/35] More complicated test case. --- .../comet/exec/CometNativeShuffleSuite.scala | 47 ++++++++++++++----- 1 file changed, 34 insertions(+), 13 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 1318f7ca5b..a97baf8ec1 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -19,19 +19,17 @@ package org.apache.comet.exec -import scala.concurrent.duration.DurationInt - -import org.scalactic.source.Position -import org.scalatest.Tag - +import org.apache.comet.CometConf import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv -import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} +import org.scalactic.source.Position +import org.scalatest.Tag -import org.apache.comet.CometConf +import scala.concurrent.duration.DurationInt class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit @@ -251,15 +249,38 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper // This duplicates behavior seen in a much more complicated Spark SQL test // "SPARK-44647: test join key is the second cluster key" test("range partitioning with duplicate column references") { - withParquetTable((0 until 100).map(i => (i % 10, (i % 5).toByte, i % 3)), "tbl") { + // Test with wider schema and non-adjacent duplicate columns + withParquetTable( + (0 until 100).map(i => (i % 10, (i % 5).toByte, i % 3, i % 7, (i % 4).toShort, i.toString)), + "tbl") { val df = sql("SELECT * FROM tbl") - // This is the problematic case: duplicate references to the same column - // Both _1 references should get different BoundReference indices - val rangePartitioned = df.repartitionByRange(3, df.col("_1"), df.col("_1"), df.col("_2")) - - checkShuffleAnswer(rangePartitioned, 1) + // Test case 1: Adjacent duplicates (original case) + val rangePartitioned1 = df.repartitionByRange(3, df.col("_1"), df.col("_1"), df.col("_2")) + checkShuffleAnswer(rangePartitioned1, 1) + + // Test case 2: Non-adjacent duplicates in wider schema + // Duplicate _1 at positions 0 and 3, with different columns in between + val rangePartitioned2 = + df.repartitionByRange(4, df.col("_1"), df.col("_3"), df.col("_5"), df.col("_1")) + checkShuffleAnswer(rangePartitioned2, 1) + + // Test case 3: Multiple duplicate pairs + // _1 duplicated at positions 0,2 and _4 duplicated at positions 1,3 + val rangePartitioned3 = + df.repartitionByRange(4, df.col("_1"), df.col("_4"), df.col("_1"), df.col("_4")) + checkShuffleAnswer(rangePartitioned3, 1) + + // Test case 4: Triple duplicates with gaps + val rangePartitioned4 = df.repartitionByRange( + 5, + df.col("_1"), + df.col("_2"), + df.col("_1"), + df.col("_3"), + df.col("_1")) + checkShuffleAnswer(rangePartitioned4, 1) } } From 3a83258044d8e47ca3495e5d191f719c1a1db106 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 19 Sep 2025 17:35:55 -0400 Subject: [PATCH 21/35] More complicated test case. --- .../comet/exec/CometNativeShuffleSuite.scala | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index a97baf8ec1..c3d9c2b3da 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -19,21 +19,23 @@ package org.apache.comet.exec -import org.apache.comet.CometConf +import scala.concurrent.duration.DurationInt + +import org.scalactic.source.Position +import org.scalatest.Tag + import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col -import org.apache.spark.sql.{CometTestBase, DataFrame, Row} -import org.scalactic.source.Position -import org.scalatest.Tag -import scala.concurrent.duration.DurationInt +import org.apache.comet.CometConf class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { + pos: Position): Unit = { super.test(testName, testTags: _*) { withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", @@ -341,9 +343,9 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper * used by `df` are Comet native operators. */ private def checkShuffleAnswer( - df: DataFrame, - expectedNum: Int, - checkNativeOperators: Boolean = false): Unit = { + df: DataFrame, + expectedNum: Int, + checkNativeOperators: Boolean = false): Unit = { checkCometExchange(df, expectedNum, true) if (checkNativeOperators) { checkSparkAnswerAndOperator(df) From a1adf7f94afae2f5cfcb751cc04025cd7c6b2e8c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 19 Sep 2025 17:36:16 -0400 Subject: [PATCH 22/35] Test fix for duplicate expression references in range partitioning. --- .../shuffle/CometNativeShuffleWriter.scala | 44 +++++++++++++++---- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 090547c0ed..6e6aba703b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -22,6 +22,7 @@ package org.apache.spark.sql.comet.execution.shuffle import java.nio.{ByteBuffer, ByteOrder} import java.nio.file.{Files, Paths} +import scala.collection.mutable import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkEnv, TaskContext} @@ -29,7 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriter} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, SinglePartition} import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric @@ -218,6 +219,23 @@ class CometNativeShuffleWriter[K, V]( val partitioning = PartitioningOuterClass.RangePartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) + // Detect duplicates by tracking bound references to same exprId + // DataFusion will deduplicate identical sort expressions in LexOrdering, + // so we need to transform boundary rows to match the deduplicated structure + val seenExprIds = mutable.HashSet[Long]() + val deduplicationMap = mutable.ArrayBuffer[(Int, Boolean)]() // (originalIndex, isKept) + + rangePartitioning.ordering.zipWithIndex.foreach { case (sortOrder, idx) => + val attr = sortOrder.child.asInstanceOf[AttributeReference] + + if (seenExprIds.contains(attr.exprId.id)) { + deduplicationMap += (idx -> false) // Will be deduplicated by DataFusion + } else { + seenExprIds += attr.exprId.id + deduplicationMap += (idx -> true) // Will be kept by DataFusion + } + } + { // Serialize the ordering expressions for comparisons val orderingExprs = rangePartitioning.ordering @@ -234,16 +252,26 @@ class CometNativeShuffleWriter[K, V]( // internal entry is a value in that row. In other words, these are stored in row major // order, not column major val boundarySchema = rangePartitioning.ordering.flatMap(e => Some(e.dataType)) - val boundaryExprs: Seq[Seq[Literal]] = - rangePartitionBounds.get.map((row: InternalRow) => - // For every InternalRow, map its values to Literals to ao collection of Literals - row.toSeq(boundarySchema).zip(boundarySchema).map { case (value, valueType) => - Literal(value, valueType) - }) + + // Transform boundary rows to match DataFusion's deduplicated structure + val transformedBoundaryExprs: Seq[Seq[Literal]] = + rangePartitionBounds.get.map((row: InternalRow) => { + // For every InternalRow, map its values to Literals + val allLiterals = + row.toSeq(boundarySchema).zip(boundarySchema).map { case (value, valueType) => + Literal(value, valueType) + } + + // Keep only the literals that correspond to non-deduplicated expressions + allLiterals + .zip(deduplicationMap) + .filter(_._2._2) // Keep only where isKept = true + .map(_._1) // Extract the literal + }) { // Convert the sequences of Literals to a collection of serialized BoundaryRows - val boundaryRows: Seq[PartitioningOuterClass.BoundaryRow] = boundaryExprs + val boundaryRows: Seq[PartitioningOuterClass.BoundaryRow] = transformedBoundaryExprs .map((rowLiterals: Seq[Literal]) => { // Serialize each sequence of Literals as a BoundaryRow val rowBuilder = PartitioningOuterClass.BoundaryRow.newBuilder(); From f287dd17b6e3ce86104f834ebafd58431bfd53b6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 07:46:06 -0400 Subject: [PATCH 23/35] New plans? --- .../q1/simplified.txt | 12 ++--- .../q11/simplified.txt | 16 +++--- .../q12/simplified.txt | 4 +- .../q13/simplified.txt | 4 +- .../q14a/simplified.txt | 20 +++---- .../q14b/simplified.txt | 12 ++--- .../q15/simplified.txt | 4 +- .../q16/simplified.txt | 2 +- .../q17/simplified.txt | 4 +- .../q18/simplified.txt | 4 +- .../q19/simplified.txt | 4 +- .../q2/simplified.txt | 6 +-- .../q20/simplified.txt | 4 +- .../q21/simplified.txt | 4 +- .../q22/simplified.txt | 4 +- .../q23a/simplified.txt | 20 +++---- .../q23b/simplified.txt | 26 +++++----- .../q25/simplified.txt | 4 +- .../q26/simplified.txt | 4 +- .../q27/simplified.txt | 4 +- .../q29/simplified.txt | 4 +- .../q3/simplified.txt | 4 +- .../q30/simplified.txt | 12 ++--- .../q31/explain.txt | 6 +-- .../q31/simplified.txt | 26 +++++----- .../q32/simplified.txt | 8 +-- .../q33/simplified.txt | 16 +++--- .../q34/explain.txt | 6 +-- .../q34/simplified.txt | 4 +- .../q36/simplified.txt | 4 +- .../q38/simplified.txt | 2 +- .../q39a/simplified.txt | 8 +-- .../q39b/simplified.txt | 8 +-- .../q4/simplified.txt | 24 ++++----- .../q40/simplified.txt | 4 +- .../q41/simplified.txt | 2 +- .../q42/simplified.txt | 4 +- .../q43/simplified.txt | 4 +- .../q44/simplified.txt | 10 ++-- .../q46/simplified.txt | 4 +- .../q47/simplified.txt | 6 +-- .../q48/simplified.txt | 4 +- .../q49/simplified.txt | 12 ++--- .../q5/simplified.txt | 6 +-- .../q50/simplified.txt | 4 +- .../q51/simplified.txt | 8 +-- .../q52/simplified.txt | 4 +- .../q53/simplified.txt | 4 +- .../q54/simplified.txt | 6 +-- .../q55/simplified.txt | 4 +- .../q56/simplified.txt | 16 +++--- .../q57/simplified.txt | 6 +-- .../q58/simplified.txt | 12 ++--- .../q59/simplified.txt | 6 +-- .../q6/simplified.txt | 6 +-- .../q60/simplified.txt | 16 +++--- .../q61/simplified.txt | 8 +-- .../q62/simplified.txt | 4 +- .../q63/simplified.txt | 4 +- .../q64/explain.txt | 6 +-- .../q64/simplified.txt | 16 +++--- .../q65/simplified.txt | 12 ++--- .../q66/simplified.txt | 12 ++--- .../q67/simplified.txt | 4 +- .../q68/simplified.txt | 4 +- .../q7/simplified.txt | 4 +- .../q70/simplified.txt | 4 +- .../q71/simplified.txt | 4 +- .../q72/simplified.txt | 2 +- .../q73/simplified.txt | 2 +- .../q74/simplified.txt | 16 +++--- .../q75/simplified.txt | 8 +-- .../q76/simplified.txt | 4 +- .../q77/simplified.txt | 24 ++++----- .../q78/simplified.txt | 12 ++--- .../q79/simplified.txt | 4 +- .../q8/simplified.txt | 6 +-- .../q80/simplified.txt | 6 +-- .../q81/simplified.txt | 12 ++--- .../q83/simplified.txt | 12 ++--- .../q85/simplified.txt | 4 +- .../q86/simplified.txt | 4 +- .../q88/simplified.txt | 16 +++--- .../q89/simplified.txt | 4 +- .../q9/simplified.txt | 20 +++---- .../q90/simplified.txt | 4 +- .../q91/simplified.txt | 4 +- .../q92/simplified.txt | 8 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 2 +- .../q95/simplified.txt | 2 +- .../q96/simplified.txt | 2 +- .../q97/simplified.txt | 4 +- .../q98/simplified.txt | 4 +- .../q99/simplified.txt | 4 +- .../q1/simplified.txt | 4 +- .../q11/simplified.txt | 8 +-- .../q12/simplified.txt | 2 +- .../q13/simplified.txt | 2 +- .../q14a/simplified.txt | 20 +++---- .../q14b/simplified.txt | 12 ++--- .../q15/simplified.txt | 2 +- .../q16/simplified.txt | 2 +- .../q17/simplified.txt | 4 +- .../q18/simplified.txt | 4 +- .../q19/simplified.txt | 2 +- .../q2/simplified.txt | 2 +- .../q20/simplified.txt | 2 +- .../q21/simplified.txt | 4 +- .../q22/simplified.txt | 4 +- .../q23a/simplified.txt | 20 +++---- .../q23b/simplified.txt | 26 +++++----- .../q25/simplified.txt | 2 +- .../q26/simplified.txt | 4 +- .../q27/simplified.txt | 4 +- .../q29/simplified.txt | 4 +- .../q3/simplified.txt | 2 +- .../q30/simplified.txt | 4 +- .../q31/simplified.txt | 12 ++--- .../q32/simplified.txt | 6 +-- .../q33/simplified.txt | 6 +-- .../q34/explain.txt | 6 +-- .../q34/simplified.txt | 4 +- .../q36/simplified.txt | 2 +- .../q38/simplified.txt | 2 +- .../q39a/simplified.txt | 8 +-- .../q39b/simplified.txt | 8 +-- .../q4/simplified.txt | 24 ++++----- .../q40/simplified.txt | 4 +- .../q41/simplified.txt | 2 +- .../q42/simplified.txt | 2 +- .../q43/simplified.txt | 2 +- .../q44/simplified.txt | 10 ++-- .../q46/simplified.txt | 2 +- .../q47/simplified.txt | 2 +- .../q48/simplified.txt | 4 +- .../q49/simplified.txt | 12 ++--- .../q5/simplified.txt | 6 +-- .../q50/simplified.txt | 4 +- .../q51/simplified.txt | 4 +- .../q52/simplified.txt | 2 +- .../q53/simplified.txt | 2 +- .../q54/simplified.txt | 2 +- .../q55/simplified.txt | 2 +- .../q56/simplified.txt | 6 +-- .../q57/simplified.txt | 2 +- .../q58/simplified.txt | 6 +-- .../q59/simplified.txt | 2 +- .../q6/simplified.txt | 6 +-- .../q60/simplified.txt | 6 +-- .../q61/simplified.txt | 4 +- .../q62/simplified.txt | 4 +- .../q63/simplified.txt | 2 +- .../q64/explain.txt | 6 +-- .../q64/simplified.txt | 4 +- .../q65/simplified.txt | 4 +- .../q66/simplified.txt | 12 ++--- .../q67/simplified.txt | 4 +- .../q68/simplified.txt | 2 +- .../q7/simplified.txt | 4 +- .../q70/simplified.txt | 2 +- .../q71/simplified.txt | 2 +- .../q72/simplified.txt | 2 +- .../q73/simplified.txt | 2 +- .../q74/simplified.txt | 8 +-- .../q75/simplified.txt | 4 +- .../q76/simplified.txt | 2 +- .../q77/simplified.txt | 12 ++--- .../q78/simplified.txt | 6 +-- .../q79/simplified.txt | 2 +- .../q8/simplified.txt | 4 +- .../q80/simplified.txt | 6 +-- .../q81/simplified.txt | 4 +- .../q83.ansi/simplified.txt | 12 ++--- .../q85/simplified.txt | 4 +- .../q86/simplified.txt | 2 +- .../q88/simplified.txt | 16 +++--- .../q89/simplified.txt | 2 +- .../q9/simplified.txt | 20 +++---- .../q90/simplified.txt | 4 +- .../q91/simplified.txt | 2 +- .../q92/simplified.txt | 6 +-- .../q93/simplified.txt | 4 +- .../q94/simplified.txt | 2 +- .../q95/simplified.txt | 2 +- .../q96/simplified.txt | 2 +- .../q97/simplified.txt | 4 +- .../q98/simplified.txt | 2 +- .../q99/simplified.txt | 4 +- .../approved-plans-v1_4/q1/simplified.txt | 12 ++--- .../approved-plans-v1_4/q11/simplified.txt | 16 +++--- .../approved-plans-v1_4/q12/simplified.txt | 4 +- .../approved-plans-v1_4/q13/simplified.txt | 4 +- .../approved-plans-v1_4/q14a/simplified.txt | 20 +++---- .../approved-plans-v1_4/q14b/simplified.txt | 12 ++--- .../approved-plans-v1_4/q15/simplified.txt | 4 +- .../approved-plans-v1_4/q16/simplified.txt | 2 +- .../approved-plans-v1_4/q17/simplified.txt | 4 +- .../approved-plans-v1_4/q18/simplified.txt | 4 +- .../approved-plans-v1_4/q19/simplified.txt | 4 +- .../approved-plans-v1_4/q2/simplified.txt | 6 +-- .../approved-plans-v1_4/q20/simplified.txt | 4 +- .../approved-plans-v1_4/q21/simplified.txt | 4 +- .../approved-plans-v1_4/q22/simplified.txt | 4 +- .../approved-plans-v1_4/q23a/simplified.txt | 20 +++---- .../approved-plans-v1_4/q23b/simplified.txt | 26 +++++----- .../approved-plans-v1_4/q25/simplified.txt | 4 +- .../approved-plans-v1_4/q26/simplified.txt | 4 +- .../approved-plans-v1_4/q27/simplified.txt | 4 +- .../approved-plans-v1_4/q29/simplified.txt | 4 +- .../approved-plans-v1_4/q3/simplified.txt | 4 +- .../approved-plans-v1_4/q30/simplified.txt | 12 ++--- .../approved-plans-v1_4/q31/explain.txt | 6 +-- .../approved-plans-v1_4/q31/simplified.txt | 26 +++++----- .../approved-plans-v1_4/q32/simplified.txt | 8 +-- .../approved-plans-v1_4/q33/simplified.txt | 16 +++--- .../approved-plans-v1_4/q34/explain.txt | 6 +-- .../approved-plans-v1_4/q34/simplified.txt | 4 +- .../approved-plans-v1_4/q36/simplified.txt | 4 +- .../approved-plans-v1_4/q38/simplified.txt | 2 +- .../approved-plans-v1_4/q39a/simplified.txt | 8 +-- .../approved-plans-v1_4/q39b/simplified.txt | 8 +-- .../approved-plans-v1_4/q4/simplified.txt | 24 ++++----- .../approved-plans-v1_4/q40/simplified.txt | 4 +- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q42/simplified.txt | 4 +- .../approved-plans-v1_4/q43/simplified.txt | 4 +- .../approved-plans-v1_4/q44/simplified.txt | 8 +-- .../approved-plans-v1_4/q46/simplified.txt | 4 +- .../approved-plans-v1_4/q47/simplified.txt | 6 +-- .../approved-plans-v1_4/q48/simplified.txt | 4 +- .../approved-plans-v1_4/q49/simplified.txt | 12 ++--- .../approved-plans-v1_4/q5/simplified.txt | 6 +-- .../approved-plans-v1_4/q50/simplified.txt | 4 +- .../approved-plans-v1_4/q51/simplified.txt | 8 +-- .../approved-plans-v1_4/q52/simplified.txt | 4 +- .../approved-plans-v1_4/q53/simplified.txt | 4 +- .../approved-plans-v1_4/q54/simplified.txt | 6 +-- .../approved-plans-v1_4/q55/simplified.txt | 4 +- .../approved-plans-v1_4/q56/simplified.txt | 16 +++--- .../approved-plans-v1_4/q57/simplified.txt | 6 +-- .../approved-plans-v1_4/q58/simplified.txt | 12 ++--- .../approved-plans-v1_4/q59/simplified.txt | 6 +-- .../approved-plans-v1_4/q6/simplified.txt | 6 +-- .../approved-plans-v1_4/q60/simplified.txt | 16 +++--- .../approved-plans-v1_4/q61/simplified.txt | 8 +-- .../approved-plans-v1_4/q62/simplified.txt | 4 +- .../approved-plans-v1_4/q63/simplified.txt | 4 +- .../approved-plans-v1_4/q64/explain.txt | 6 +-- .../approved-plans-v1_4/q64/simplified.txt | 16 +++--- .../approved-plans-v1_4/q65/simplified.txt | 12 ++--- .../approved-plans-v1_4/q66/simplified.txt | 12 ++--- .../approved-plans-v1_4/q67/simplified.txt | 4 +- .../approved-plans-v1_4/q68/simplified.txt | 4 +- .../approved-plans-v1_4/q7/simplified.txt | 4 +- .../approved-plans-v1_4/q70/simplified.txt | 4 +- .../approved-plans-v1_4/q71/simplified.txt | 4 +- .../approved-plans-v1_4/q72/simplified.txt | 2 +- .../approved-plans-v1_4/q73/simplified.txt | 2 +- .../approved-plans-v1_4/q74/simplified.txt | 16 +++--- .../approved-plans-v1_4/q75/simplified.txt | 8 +-- .../approved-plans-v1_4/q76/simplified.txt | 4 +- .../approved-plans-v1_4/q77/simplified.txt | 24 ++++----- .../approved-plans-v1_4/q78/simplified.txt | 12 ++--- .../approved-plans-v1_4/q79/simplified.txt | 4 +- .../approved-plans-v1_4/q8/simplified.txt | 6 +-- .../approved-plans-v1_4/q80/simplified.txt | 6 +-- .../approved-plans-v1_4/q81/simplified.txt | 12 ++--- .../approved-plans-v1_4/q83/simplified.txt | 12 ++--- .../approved-plans-v1_4/q85/simplified.txt | 4 +- .../approved-plans-v1_4/q86/simplified.txt | 4 +- .../approved-plans-v1_4/q88/simplified.txt | 16 +++--- .../approved-plans-v1_4/q89/simplified.txt | 4 +- .../approved-plans-v1_4/q9/simplified.txt | 20 +++---- .../approved-plans-v1_4/q90/simplified.txt | 4 +- .../approved-plans-v1_4/q91/simplified.txt | 4 +- .../approved-plans-v1_4/q92/simplified.txt | 8 +-- .../approved-plans-v1_4/q93/simplified.txt | 4 +- .../approved-plans-v1_4/q94/simplified.txt | 2 +- .../approved-plans-v1_4/q95/simplified.txt | 2 +- .../approved-plans-v1_4/q96/simplified.txt | 2 +- .../approved-plans-v1_4/q97/simplified.txt | 4 +- .../approved-plans-v1_4/q98/simplified.txt | 4 +- .../approved-plans-v1_4/q99/simplified.txt | 4 +- .../q10a/simplified.txt | 2 +- .../q11/simplified.txt | 16 +++--- .../q12/simplified.txt | 4 +- .../q14/simplified.txt | 12 ++--- .../q14a/simplified.txt | 44 ++++++++-------- .../q18a/simplified.txt | 20 +++---- .../q20/simplified.txt | 4 +- .../q22a/simplified.txt | 32 ++++++------ .../q27a/simplified.txt | 12 ++--- .../q34/explain.txt | 6 +-- .../q34/simplified.txt | 4 +- .../q35a/simplified.txt | 2 +- .../q36a/simplified.txt | 18 +++---- .../q47/simplified.txt | 6 +-- .../q49/simplified.txt | 12 ++--- .../q51a/simplified.txt | 8 +-- .../q57/simplified.txt | 6 +-- .../q5a/simplified.txt | 6 +-- .../q6/simplified.txt | 6 +-- .../q64/explain.txt | 6 +-- .../q64/simplified.txt | 16 +++--- .../q67a/simplified.txt | 52 +++++++++---------- .../q70a/simplified.txt | 6 +-- .../q72/simplified.txt | 2 +- .../q74/simplified.txt | 16 +++--- .../q75/simplified.txt | 8 +-- .../q77a/simplified.txt | 24 ++++----- .../q78/simplified.txt | 12 ++--- .../q80a/simplified.txt | 6 +-- .../q86a/simplified.txt | 18 +++---- .../q98/simplified.txt | 4 +- .../q10a/simplified.txt | 2 +- .../q11/simplified.txt | 8 +-- .../q12/simplified.txt | 2 +- .../q14/simplified.txt | 12 ++--- .../q14a/simplified.txt | 44 ++++++++-------- .../q18a/simplified.txt | 20 +++---- .../q20/simplified.txt | 2 +- .../q22a/simplified.txt | 32 ++++++------ .../q27a/simplified.txt | 12 ++--- .../q34/explain.txt | 6 +-- .../q34/simplified.txt | 4 +- .../q35a/simplified.txt | 2 +- .../q36a/simplified.txt | 4 +- .../q47/simplified.txt | 2 +- .../q49/simplified.txt | 12 ++--- .../q51a/simplified.txt | 4 +- .../q57/simplified.txt | 2 +- .../q5a/simplified.txt | 6 +-- .../q6/simplified.txt | 6 +-- .../q64/explain.txt | 6 +-- .../q64/simplified.txt | 4 +- .../q67a/simplified.txt | 52 +++++++++---------- .../q70a/simplified.txt | 4 +- .../q72/simplified.txt | 2 +- .../q74/simplified.txt | 8 +-- .../q75/simplified.txt | 4 +- .../q77a/simplified.txt | 12 ++--- .../q78/simplified.txt | 6 +-- .../q80a/simplified.txt | 6 +-- .../q86a/simplified.txt | 4 +- .../q98/simplified.txt | 2 +- .../approved-plans-v2_7/q10a/simplified.txt | 2 +- .../approved-plans-v2_7/q11/simplified.txt | 16 +++--- .../approved-plans-v2_7/q12/simplified.txt | 4 +- .../approved-plans-v2_7/q14/simplified.txt | 12 ++--- .../approved-plans-v2_7/q14a/simplified.txt | 44 ++++++++-------- .../approved-plans-v2_7/q18a/simplified.txt | 20 +++---- .../approved-plans-v2_7/q20/simplified.txt | 4 +- .../approved-plans-v2_7/q22a/simplified.txt | 32 ++++++------ .../approved-plans-v2_7/q27a/simplified.txt | 12 ++--- .../approved-plans-v2_7/q34/explain.txt | 6 +-- .../approved-plans-v2_7/q34/simplified.txt | 4 +- .../approved-plans-v2_7/q35a/simplified.txt | 2 +- .../approved-plans-v2_7/q36a/simplified.txt | 18 +++---- .../approved-plans-v2_7/q47/simplified.txt | 6 +-- .../approved-plans-v2_7/q49/simplified.txt | 12 ++--- .../approved-plans-v2_7/q51a/simplified.txt | 8 +-- .../approved-plans-v2_7/q57/simplified.txt | 6 +-- .../approved-plans-v2_7/q5a/simplified.txt | 6 +-- .../approved-plans-v2_7/q6/simplified.txt | 6 +-- .../approved-plans-v2_7/q64/explain.txt | 6 +-- .../approved-plans-v2_7/q64/simplified.txt | 16 +++--- .../approved-plans-v2_7/q67a/simplified.txt | 52 +++++++++---------- .../approved-plans-v2_7/q70a/simplified.txt | 6 +-- .../approved-plans-v2_7/q72/simplified.txt | 2 +- .../approved-plans-v2_7/q74/simplified.txt | 16 +++--- .../approved-plans-v2_7/q75/simplified.txt | 8 +-- .../approved-plans-v2_7/q77a/simplified.txt | 24 ++++----- .../approved-plans-v2_7/q78/simplified.txt | 12 ++--- .../approved-plans-v2_7/q80a/simplified.txt | 6 +-- .../approved-plans-v2_7/q86a/simplified.txt | 18 +++---- .../approved-plans-v2_7/q98/simplified.txt | 4 +- 377 files changed, 1502 insertions(+), 1502 deletions(-) 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 2a9cd830fb..3e1d6243f8 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_store_sk] CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] @@ -30,12 +30,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return] - CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] + CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] 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 a3bacdf562..4599ecf539 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -33,9 +33,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -58,9 +58,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -74,9 +74,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 0a1a7d8020..fb83fd2f9a 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 a1a0200121..8ef882a435 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,9 +1,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 - CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] 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] 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 13c79b4299..79c782f2ca 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] @@ -13,9 +13,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -116,9 +116,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -133,9 +133,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 5d5557bc59..fb9abae378 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -119,9 +119,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 6eac0d3bd7..c39b96efe3 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] CometExchange [ca_zip] #1 - CometHashAggregate [ca_zip,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [ca_zip,sum] 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] 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 352d9e22b1..c703a75e78 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] 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] 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 3a978f4afb..1d48f96b09 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [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(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))] CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] 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] 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 bab3eee57c..6c2b8b2e4f 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)))] + CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,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)))] CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] 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 94d8c234cb..93ab89c142 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] 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] 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 420fd43d85..852c5fca0f 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [d_week_seq] #2 - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] 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] @@ -28,7 +28,7 @@ WholeStageCodegen (1) CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 CometBroadcastExchange [d_week_seq] #6 CometProject [d_week_seq] 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 fdf7f7ffa6..2958d060fe 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 7583431e56..1c2e80c991 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [w_warehouse_name,i_item_id,inv_before,inv_after,sum,sum,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] + CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,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)] CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] 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] 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 acf94de869..9119ee749d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] 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] 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 cd74cced17..8f1bddf6c0 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] CometExchange #1 - CometHashAggregate [sum,isEmpty,sales] + CometHashAggregate [sales] [sum,isEmpty] CometUnion [sales] CometProject [cs_quantity,cs_list_price] [sales] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -56,12 +56,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -81,9 +81,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -111,6 +111,6 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt index 0575c4a672..92563114ae 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] @@ -28,7 +28,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -59,12 +59,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -84,9 +84,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -106,15 +106,15 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] @@ -132,7 +132,7 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt index c2693d0fb5..dcad304452 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 dd6b1c575b..0b56a47547 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 47dd8a4f50..bdae0cc477 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] 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] 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 e8fa7836e1..15b992d3a4 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 f28011f206..7bdad4b94f 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 e4b6a5276e..0227534fa5 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_total_return] CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] @@ -36,12 +36,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] CometExchange [ctr_state] #6 - CometHashAggregate [ctr_state,sum,count,ctr_total_return] - CometHashAggregate [ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] 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 4f065b40bd..83d717e295 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] 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 435da991de..2f97384320 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] @@ -11,9 +11,9 @@ WholeStageCodegen (1) CometProject [ca_county,d_year,store_sales,store_sales,store_sales] CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [ca_county,d_year,store_sales,d_qoy,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -34,9 +34,9 @@ WholeStageCodegen (1) CometFilter [ca_address_sk,ca_county] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -55,9 +55,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -76,9 +76,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -89,9 +89,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year,d_qoy] #4 ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -102,9 +102,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year,d_qoy] #9 ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] 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 a89a381578..07619fc999 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] + CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 - CometHashAggregate [sum,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [sum] CometProject [cs_ext_discount_amt] CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_ext_discount_amt,cs_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count,avg(UnscaledValue(cs_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] 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] 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 0628508d3f..b4b84c2ac6 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] CometExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] CometUnion [i_manufact_id,total_sales] - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_manufact_id] #2 - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] 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] @@ -41,9 +41,9 @@ WholeStageCodegen (1) CometProject [i_manufact_id] CometFilter [i_category,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_manufact_id] #8 - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] 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] @@ -56,9 +56,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #4 ReusedExchange [ca_address_sk] #5 ReusedExchange [i_item_sk,i_manufact_id] #6 - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_manufact_id] #9 - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] 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] 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 ceab9d3f0a..287e323952 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 a7c1215d1c..47d0e35dd2 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 d7926e49ee..61782e3011 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 - CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] 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] 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 c9ecc235c5..7ceccf5d2c 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 4a5376e4f7..2cd1f70c8a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 4a5376e4f7..2cd1f70c8a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 47e4022f45..36be965b48 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 @@ -12,9 +12,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -36,9 +36,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -77,9 +77,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -94,9 +94,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year] #9 CometBroadcastExchange [customer_id,year_total] #16 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] @@ -110,9 +110,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] 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 23945dce83..79e3556750 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,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)] CometExchange [w_state,i_item_id] #1 - CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty,d_date,cs_sales_price,cr_refunded_cash] + CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] 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] 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 bff847188c..99c935aaa4 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 @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometBroadcastExchange [i_manufact] #2 CometProject [i_manufact] CometFilter [item_cnt,i_manufact] - CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] CometExchange [i_manufact] #3 CometHashAggregate [i_manufact,count] CometProject [i_manufact] [i_manufact] 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 9de0f79dee..839937d40c 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] 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] 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 c7445ea60c..59ad7611a5 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] 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] 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 018f1919e4..d0c3ad435b 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 @@ -29,15 +29,15 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] CometProject [ss_store_sk,ss_net_profit] CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] @@ -61,7 +61,7 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] 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))] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] ReusedExchange [ss_item_sk,sum,count] #2 InputAdapter BroadcastExchange #5 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 ab17b13248..e646cb620a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ss_addr_sk,ca_city,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] 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] 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 71df9be678..19124a9e20 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 b456a3d4a3..e29e101c7e 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] + CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] CometExchange #1 - CometHashAggregate [sum,ss_quantity] + CometHashAggregate [ss_quantity] [sum] CometProject [ss_quantity] CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 b967ae28f0..e86e0869f5 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [web_site_id] #8 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 a85fe8c06f..98a44eaf40 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] 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] 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 30c94463ad..a5af2e114b 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 @@ -21,9 +21,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -51,9 +51,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 18bcdb314f..218062c3ef 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 b8716d034f..0c123eb728 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,sum_sales,_w0,d_qoy,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] 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] 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 ea21e4f051..ae0cecb1ba 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 @@ -2,12 +2,12 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [segment,num_customers,segment_base,count,count(1)] + CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] CometExchange [segment] #1 CometHashAggregate [segment,count] - CometHashAggregate [segment,c_customer_sk,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [c_customer_sk] #2 - CometHashAggregate [c_customer_sk,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] 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] 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 dde0a8ebc7..dd7658cbac 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] 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] 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 88b9477d1d..19eca782cd 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] CometUnion [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -42,9 +42,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_id] CometFilter [i_item_id,i_color] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #4 ReusedExchange [ca_address_sk] #5 ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 b131b85ea9..b854e818db 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 1cc3e6bfff..96e11bc0c7 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometProject [item_id,ss_item_rev,cs_item_rev] CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] CometFilter [item_id,ss_item_rev] - CometHashAggregate [item_id,ss_item_rev,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -48,9 +48,9 @@ WholeStageCodegen (1) ReusedExchange [d_date] #3 CometBroadcastExchange [item_id,cs_item_rev] #6 CometFilter [item_id,cs_item_rev] - CometHashAggregate [item_id,cs_item_rev,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #5 CometBroadcastExchange [item_id,ws_item_rev] #8 CometFilter [item_id,ws_item_rev] - CometHashAggregate [item_id,ws_item_rev,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 d6ad4a9650..bca9c6463d 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] 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] @@ -32,7 +32,7 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 CometBroadcastExchange [s_store_sk,s_store_id] #6 CometProject [s_store_id] [s_store_sk,s_store_id] 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 35d3166cef..2c772b5998 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -53,9 +53,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_current_price,i_category] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt index 914f53e34b..086da3b776 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] CometUnion [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -42,9 +42,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_id] CometFilter [i_item_id,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #4 ReusedExchange [ca_address_sk] #5 ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 d5d8f9b8f0..6ebe25b753 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 @@ -3,9 +3,9 @@ WholeStageCodegen (2) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] @@ -56,9 +56,9 @@ WholeStageCodegen (2) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] 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 a530489ce4..dee2e434c7 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] 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] 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 1b89499528..02166879a9 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 - CometHashAggregate [i_manager_id,sum_sales,_w0,d_moy,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] 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] 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 ed591f262b..cbe790cd00 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] 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 6f1dea6cef..43527978d8 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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 + CometExchange [product_name,store_name,cnt] #1 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] CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] @@ -117,8 +117,8 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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] CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -171,7 +171,7 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 CometBroadcastExchange [d_date_sk,d_year] #23 CometFilter [d_date_sk,d_year] 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 2e8a46fe81..cd1c84b6f2 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 @@ -12,9 +12,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [ss_store_sk,ss_item_sk,revenue,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] @@ -37,12 +37,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometBroadcastExchange [ss_store_sk,ave] #6 CometFilter [ss_store_sk,ave] - CometHashAggregate [ss_store_sk,ave,sum,count,avg(revenue)] + CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] CometExchange [ss_store_sk] #7 - CometHashAggregate [ss_store_sk,sum,count,revenue] - CometHashAggregate [ss_store_sk,revenue,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [revenue] [ss_store_sk,sum,count] + CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] 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 07fd441ba4..bc6bde24b2 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [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(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + CometHashAggregate [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] [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] CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] 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] @@ -41,9 +41,9 @@ WholeStageCodegen (1) CometProject [sm_ship_mode_sk] CometFilter [sm_ship_mode_sk,sm_carrier] CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] 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] 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 83e1e34902..9069117a5b 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 @@ -14,9 +14,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ 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))] + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] 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] 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 46feb4b4e9..fdddd82de0 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ss_addr_sk,ca_city,sum,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] + CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] 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] 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 395b135acb..a15bcd5c0d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 9d6ed26262..80f0cfc8c2 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 @@ -57,9 +57,9 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometSort [s_state,_w0] - CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #7 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 a083a4398d..63e7c353ac 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometExchange [ext_price,brand_id] #1 - CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] + CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 c8f7f27fba..227dd00f2b 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 @@ -6,7 +6,7 @@ WholeStageCodegen (1) CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 400468d3e1..72e39422e0 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 aa9d0e4801..d0520c8b63 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -82,9 +82,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 eed5e27107..b5449831cb 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] 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] 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 821894c08d..a1243769e5 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 @@ -14,9 +14,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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))] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,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] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -39,9 +39,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [s_store_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,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] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -59,9 +59,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,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] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -69,9 +69,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #4 CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,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] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -97,9 +97,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,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] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 fbe810689f..77c05217f4 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 @@ -7,9 +7,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -36,9 +36,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -57,9 +57,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ReusedExchange [d_date_sk,d_year] #5 CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 b3905b01c2..8e244d6c9b 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 @@ -4,9 +4,9 @@ WholeStageCodegen (1) CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,s_city,amt,profit,ss_addr_sk,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] 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] 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 50673ff9c1..d1542ab607 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] CometExchange [s_store_name] #1 - CometHashAggregate [s_store_name,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_store_name,sum] 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] @@ -40,7 +40,7 @@ WholeStageCodegen (1) CometBroadcastExchange [ca_zip] #7 CometProject [ca_zip] CometFilter [ca_zip,cnt] - CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] CometExchange [ca_zip] #8 CometHashAggregate [ca_zip,count] CometProject [ca_zip] 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 d403cf479b..2c56b98787 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -63,7 +63,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -96,7 +96,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 a7f793b5cb..1091e272cc 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_total_return] CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] @@ -36,12 +36,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] CometExchange [ctr_state] #6 - CometHashAggregate [ctr_state,sum,count,ctr_total_return] - CometHashAggregate [ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] 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 9c3886920e..d016a5ee2d 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] CometProject [item_id,sr_item_qty,cr_item_qty] CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [item_id,sr_item_qty,i_item_id,sum,sum(sr_return_quantity)] + CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometHashAggregate [sr_return_quantity] [i_item_id,sum] 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] @@ -43,9 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [item_id,cr_item_qty,i_item_id,sum,sum(cr_return_quantity)] + CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometHashAggregate [cr_return_quantity] [i_item_id,sum] 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] @@ -56,9 +56,9 @@ WholeStageCodegen (1) ReusedExchange [i_item_sk,i_item_id] #5 ReusedExchange [d_date_sk] #6 CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [item_id,wr_item_qty,i_item_id,sum,sum(wr_return_quantity)] + CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] CometExchange [i_item_id] #10 - CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometHashAggregate [wr_return_quantity] [i_item_id,sum] 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] 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 52cf7b57a0..dfcca32f03 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] CometExchange [r_reason_desc] #1 - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] 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] 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 92a7eb7e8a..29e2d72920 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 - CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] 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] 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 3490714189..e3923fb3ea 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 @@ -8,7 +8,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [h8_30_to_9,count,count(1)] + CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -37,7 +37,7 @@ WholeStageCodegen (8) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [h9_to_9_30,count,count(1)] + CometHashAggregate [count] [h9_to_9_30,count(1)] CometExchange #6 CometHashAggregate [count] CometProject @@ -60,7 +60,7 @@ WholeStageCodegen (8) WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [h9_30_to_10,count,count(1)] + CometHashAggregate [count] [h9_30_to_10,count(1)] CometExchange #9 CometHashAggregate [count] CometProject @@ -83,7 +83,7 @@ WholeStageCodegen (8) WholeStageCodegen (3) CometColumnarToRow InputAdapter - CometHashAggregate [h10_to_10_30,count,count(1)] + CometHashAggregate [count] [h10_to_10_30,count(1)] CometExchange #12 CometHashAggregate [count] CometProject @@ -106,7 +106,7 @@ WholeStageCodegen (8) WholeStageCodegen (4) CometColumnarToRow InputAdapter - CometHashAggregate [h10_30_to_11,count,count(1)] + CometHashAggregate [count] [h10_30_to_11,count(1)] CometExchange #15 CometHashAggregate [count] CometProject @@ -129,7 +129,7 @@ WholeStageCodegen (8) WholeStageCodegen (5) CometColumnarToRow InputAdapter - CometHashAggregate [h11_to_11_30,count,count(1)] + CometHashAggregate [count] [h11_to_11_30,count(1)] CometExchange #18 CometHashAggregate [count] CometProject @@ -152,7 +152,7 @@ WholeStageCodegen (8) WholeStageCodegen (6) CometColumnarToRow InputAdapter - CometHashAggregate [h11_30_to_12,count,count(1)] + CometHashAggregate [count] [h11_30_to_12,count(1)] CometExchange #21 CometHashAggregate [count] CometProject @@ -175,7 +175,7 @@ WholeStageCodegen (8) WholeStageCodegen (7) CometColumnarToRow InputAdapter - CometHashAggregate [h12_to_12_30,count,count(1)] + CometHashAggregate [count] [h12_to_12_30,count(1)] CometExchange #24 CometHashAggregate [count] CometProject 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 140fe297a1..61bfd1d960 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla 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 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] 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] 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 510cf60bd8..558f5f4b36 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 @@ -5,9 +5,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #1 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -18,9 +18,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #2 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -31,9 +31,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #3 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -44,9 +44,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #4 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -57,9 +57,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #5 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] 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 2db3b6c957..0991e4e0dc 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 @@ -3,7 +3,7 @@ WholeStageCodegen (2) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [amc,count,count(1)] + CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -32,7 +32,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [pmc,count,count(1)] + CometHashAggregate [count] [pmc,count(1)] CometExchange #6 CometHashAggregate [count] CometProject 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 a1b8084d84..3e9b8945da 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometExchange [Returns_Loss] #1 - CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] 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] 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 7a1825cc9b..4ec5755df7 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 - CometHashAggregate [sum,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [sum] CometProject [ws_ext_discount_amt] CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] CometProject [ws_ext_discount_amt,ws_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count,avg(UnscaledValue(ws_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] 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] 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 d761953c97..d1de4f3475 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] CometExchange [ss_customer_sk] #1 - CometHashAggregate [ss_customer_sk,sum,isEmpty,act_sales] + CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] 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] 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 068a535b6b..bf85b844c9 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 6f07040b87..3e9cdb81f5 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 36f4009885..bbbd07245e 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 49b28b94be..a7de047b25 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,9 +1,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,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 - CometHashAggregate [sum,sum,sum,customer_sk,customer_sk] + CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] CometProject [customer_sk,customer_sk] CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] CometSort [customer_sk,item_sk] 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 f563efa55d..af05a7d8c6 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 @@ -13,9 +13,9 @@ WholeStageCodegen (3) InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 2b9df4aa5a..22896ee566 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] 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] 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 be8266bd6e..0d694f3f2e 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject [c_customer_id] CometColumnarToRow InputAdapter CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] @@ -42,7 +42,7 @@ TakeOrderedAndProject [c_customer_id] CometColumnarToRow InputAdapter CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] 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 b13cb80e50..0e67122144 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -39,7 +39,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -68,7 +68,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -88,7 +88,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 80e956ac40..1fdffc3b73 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 ce0bbc078f..f7cfb4ea09 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometExchange #1 - CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] 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] 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 13c79b4299..79c782f2ca 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] @@ -13,9 +13,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -116,9 +116,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -133,9 +133,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 1c69144d59..6b5264b76d 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -121,9 +121,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 9949378214..6c4d568d44 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] CometColumnarToRow InputAdapter CometExchange [ca_zip] #1 - CometHashAggregate [ca_zip,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [ca_zip,sum] 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] 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 352d9e22b1..c703a75e78 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] 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] 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 3a978f4afb..1d48f96b09 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [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(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))] CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] 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] 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 bab3eee57c..6c2b8b2e4f 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)))] + CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,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)))] CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] 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 b41d9aee86..9b585fcb77 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] CometColumnarToRow InputAdapter CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] 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] 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 2ccab01f58..150fb01644 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 @@ -12,7 +12,7 @@ WholeStageCodegen (5) CometColumnarToRow InputAdapter CometExchange [d_week_seq] #2 - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] 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] 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 3b8f4cc6f2..db00a01c2d 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 7583431e56..1c2e80c991 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [w_warehouse_name,i_item_id,inv_before,inv_after,sum,sum,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] + CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,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)] CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] 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] 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 acf94de869..9119ee749d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] 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] 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 cd74cced17..8f1bddf6c0 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] CometExchange #1 - CometHashAggregate [sum,isEmpty,sales] + CometHashAggregate [sales] [sum,isEmpty] CometUnion [sales] CometProject [cs_quantity,cs_list_price] [sales] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -56,12 +56,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -81,9 +81,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -111,6 +111,6 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt index 0575c4a672..92563114ae 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] @@ -28,7 +28,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -59,12 +59,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -84,9 +84,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -106,15 +106,15 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] @@ -132,7 +132,7 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index d72009b6aa..2140d72398 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 dd6b1c575b..0b56a47547 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 47dd8a4f50..bdae0cc477 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] 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] 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 e8fa7836e1..15b992d3a4 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 8aba5528a8..84c696334f 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] CometColumnarToRow InputAdapter CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 a77fec8578..112d86a057 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre CometColumnarToRow InputAdapter CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] @@ -48,7 +48,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre CometColumnarToRow InputAdapter CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] 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 7d90b325e1..fcea2cb245 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 @@ -16,7 +16,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -43,7 +43,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -68,7 +68,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -93,7 +93,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -110,7 +110,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -127,7 +127,7 @@ WholeStageCodegen (7) CometColumnarToRow InputAdapter CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] 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 94b9ca0daa..e095592cc2 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometExchange #1 - CometHashAggregate [sum,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [sum] CometProject [cs_ext_discount_amt] CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_ext_discount_amt,cs_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count,avg(UnscaledValue(cs_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] 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] 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 492d5d6018..23325abbf1 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometColumnarToRow InputAdapter CometExchange [i_manufact_id] #2 - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] 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] @@ -51,7 +51,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometColumnarToRow InputAdapter CometExchange [i_manufact_id] #8 - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] 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] @@ -69,7 +69,7 @@ TakeOrderedAndProject [total_sales,i_manufact_id] CometColumnarToRow InputAdapter CometExchange [i_manufact_id] #9 - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] 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] 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 25339900c5..d8ee4f981e 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 a7c1215d1c..47d0e35dd2 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 9079b743d3..6e4a6a969b 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometColumnarToRow InputAdapter CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] 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] 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 c9ecc235c5..7ceccf5d2c 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 cd8f2c926b..e54435ae10 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 cd8f2c926b..e54435ae10 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 47e4022f45..36be965b48 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 @@ -12,9 +12,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -36,9 +36,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -77,9 +77,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -94,9 +94,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year] #9 CometBroadcastExchange [customer_id,year_total] #16 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] @@ -110,9 +110,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] 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 23945dce83..79e3556750 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,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)] CometExchange [w_state,i_item_id] #1 - CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty,d_date,cs_sales_price,cr_refunded_cash] + CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] 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] 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 bff847188c..99c935aaa4 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 @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometBroadcastExchange [i_manufact] #2 CometProject [i_manufact] CometFilter [item_cnt,i_manufact] - CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] CometExchange [i_manufact] #3 CometHashAggregate [i_manufact,count] CometProject [i_manufact] [i_manufact] 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 a84aa579c6..037bb43470 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] CometColumnarToRow InputAdapter CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] 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] 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 26c0e5a39c..ae850c9d3c 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed CometColumnarToRow InputAdapter CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] 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] 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 61ad9dd241..dba03cff02 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 @@ -31,15 +31,15 @@ WholeStageCodegen (7) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #4 - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] CometProject [ss_store_sk,ss_net_profit] CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_item_sk] #3 - CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] @@ -63,7 +63,7 @@ WholeStageCodegen (7) 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))] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] ReusedExchange [ss_item_sk,sum,count] #3 CometBroadcastExchange [i_item_sk,i_product_name] #7 CometProject [i_product_name] [i_item_sk,i_product_name] 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 cc13238e37..6754c4f519 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu CometColumnarToRow InputAdapter CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] 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] 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 ece2340551..0c8d77b89c 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra CometColumnarToRow InputAdapter CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] 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 b456a3d4a3..e29e101c7e 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] + CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] CometExchange #1 - CometHashAggregate [sum,ss_quantity] + CometHashAggregate [ss_quantity] [sum] CometProject [ss_quantity] CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 b967ae28f0..e86e0869f5 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [web_site_id] #8 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 a85fe8c06f..98a44eaf40 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] 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] 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 3749185537..dc6a7ea61f 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 @@ -26,7 +26,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometColumnarToRow InputAdapter CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -59,7 +59,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometColumnarToRow InputAdapter CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 e3e7817be1..05bbc57061 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] CometColumnarToRow InputAdapter CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 ee25b061b5..eab0efef6c 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] CometColumnarToRow InputAdapter CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] 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] 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 c5a154dc57..027968b881 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [segment,num_customers,segment_base] CometColumnarToRow InputAdapter CometExchange [c_customer_sk] #2 - CometHashAggregate [c_customer_sk,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] 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] 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 b9e1243993..428aac6c4d 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [ext_price,brand_id,brand] CometColumnarToRow InputAdapter CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] 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] 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 6226db4845..2f7c5323ee 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [total_sales,i_item_id] CometColumnarToRow InputAdapter CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -52,7 +52,7 @@ TakeOrderedAndProject [total_sales,i_item_id] CometColumnarToRow InputAdapter CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -70,7 +70,7 @@ TakeOrderedAndProject [total_sales,i_item_id] CometColumnarToRow InputAdapter CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 bc1e93baa5..dc90429f65 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ CometColumnarToRow InputAdapter CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] 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 6879208184..69bd5661b3 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 @@ -9,7 +9,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometColumnarToRow InputAdapter CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -55,7 +55,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometColumnarToRow InputAdapter CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -73,7 +73,7 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometColumnarToRow InputAdapter CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 5ed1e0408f..867d0d888e 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s CometColumnarToRow InputAdapter CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] 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] 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 98b6bac0cb..010720bc0e 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -55,9 +55,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_category,i_current_price] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] 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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index f3f83b448e..f3ab606ff1 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_item_id,total_sales] CometColumnarToRow InputAdapter CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -52,7 +52,7 @@ TakeOrderedAndProject [i_item_id,total_sales] CometColumnarToRow InputAdapter CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -70,7 +70,7 @@ TakeOrderedAndProject [i_item_id,total_sales] CometColumnarToRow InputAdapter CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 ebf21219d9..ad3a7005cc 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 @@ -5,7 +5,7 @@ WholeStageCodegen (2) CometColumnarToRow InputAdapter CometExchange #1 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] @@ -58,7 +58,7 @@ WholeStageCodegen (2) CometColumnarToRow InputAdapter CometExchange #10 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] 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 a530489ce4..dee2e434c7 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] 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] 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 8204e71996..6688a7a467 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] CometColumnarToRow InputAdapter CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] 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] 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 c098795280..44c02a061d 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (185) +- CometSort (184) - +- CometColumnarExchange (183) + +- CometExchange (183) +- CometProject (182) +- CometSortMergeJoin (181) :- CometSort (112) @@ -1013,9 +1013,9 @@ Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#1 Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -(183) CometColumnarExchange +(183) CometExchange Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (184) CometSort Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] 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 539bee5401..944f6edbde 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 @@ -2,7 +2,7 @@ WholeStageCodegen (37) 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 + CometExchange [product_name,store_name,cnt] #1 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] @@ -72,7 +72,7 @@ WholeStageCodegen (37) CometColumnarToRow InputAdapter CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] 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 e959e7e292..8f3fe39fef 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 @@ -18,7 +18,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometColumnarToRow InputAdapter CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] @@ -57,7 +57,7 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes CometColumnarToRow InputAdapter CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] 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 07fd441ba4..bc6bde24b2 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [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(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + CometHashAggregate [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] [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] CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] 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] @@ -41,9 +41,9 @@ WholeStageCodegen (1) CometProject [sm_ship_mode_sk] CometFilter [sm_ship_mode_sk,sm_carrier] CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] 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] 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 83e1e34902..9069117a5b 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 @@ -14,9 +14,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ 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))] + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] 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] 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 523fd2a441..96fde02c0a 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ CometColumnarToRow InputAdapter CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] 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] 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 395b135acb..a15bcd5c0d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 19c3d3f40e..f6953d47ff 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 @@ -59,7 +59,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometExchange [s_state] #7 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 668be5d815..45ecc07450 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) CometColumnarToRow InputAdapter CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 c8f7f27fba..227dd00f2b 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 @@ -6,7 +6,7 @@ WholeStageCodegen (1) CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 63fbde50df..58b40d0c50 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -38,7 +38,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -67,7 +67,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -87,7 +87,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 94b40c8419..9a3700b0b1 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 @@ -12,7 +12,7 @@ WholeStageCodegen (3) CometColumnarToRow InputAdapter CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -90,7 +90,7 @@ WholeStageCodegen (3) CometColumnarToRow InputAdapter CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 6f46a53974..5182643eac 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_ CometColumnarToRow InputAdapter CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] 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] 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 fe567c8ee8..10c3b0f213 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [s_store_sk] #2 - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -45,7 +45,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [s_store_sk] #7 - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -65,7 +65,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -75,7 +75,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange #10 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -88,7 +88,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [wp_web_page_sk] #11 - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -107,7 +107,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 3d76d5e33a..761e18b0a1 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometColumnarToRow InputAdapter CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -46,7 +46,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometColumnarToRow InputAdapter CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -71,7 +71,7 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometColumnarToRow InputAdapter CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 e9ca8f3cf2..93043d636b 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 @@ -6,7 +6,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, CometColumnarToRow InputAdapter CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] 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] 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 e8bc23fd1b..5d50f322d5 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 @@ -4,7 +4,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometColumnarToRow InputAdapter CometExchange [s_store_name] #1 - CometHashAggregate [s_store_name,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_store_name,sum] 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] @@ -40,7 +40,7 @@ TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] CometBroadcastExchange [ca_zip] #7 CometProject [ca_zip] CometFilter [ca_zip,cnt] - CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] CometExchange [ca_zip] #8 CometHashAggregate [ca_zip,count] CometProject [ca_zip] 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 d403cf479b..2c56b98787 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -63,7 +63,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -96,7 +96,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 0f58d31d35..2d929eed96 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st CometColumnarToRow InputAdapter CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] @@ -48,7 +48,7 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st CometColumnarToRow InputAdapter CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] 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 9c3886920e..d016a5ee2d 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] CometProject [item_id,sr_item_qty,cr_item_qty] CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [item_id,sr_item_qty,i_item_id,sum,sum(sr_return_quantity)] + CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometHashAggregate [sr_return_quantity] [i_item_id,sum] 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] @@ -43,9 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [item_id,cr_item_qty,i_item_id,sum,sum(cr_return_quantity)] + CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometHashAggregate [cr_return_quantity] [i_item_id,sum] 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] @@ -56,9 +56,9 @@ WholeStageCodegen (1) ReusedExchange [i_item_sk,i_item_id] #5 ReusedExchange [d_date_sk] #6 CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [item_id,wr_item_qty,i_item_id,sum,sum(wr_return_quantity)] + CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] CometExchange [i_item_id] #10 - CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometHashAggregate [wr_return_quantity] [i_item_id,sum] 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] 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 52cf7b57a0..dfcca32f03 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] CometExchange [r_reason_desc] #1 - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] 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] 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 c82a723a23..539b436013 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometColumnarToRow InputAdapter CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] 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] 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 3490714189..e3923fb3ea 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 @@ -8,7 +8,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [h8_30_to_9,count,count(1)] + CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -37,7 +37,7 @@ WholeStageCodegen (8) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [h9_to_9_30,count,count(1)] + CometHashAggregate [count] [h9_to_9_30,count(1)] CometExchange #6 CometHashAggregate [count] CometProject @@ -60,7 +60,7 @@ WholeStageCodegen (8) WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [h9_30_to_10,count,count(1)] + CometHashAggregate [count] [h9_30_to_10,count(1)] CometExchange #9 CometHashAggregate [count] CometProject @@ -83,7 +83,7 @@ WholeStageCodegen (8) WholeStageCodegen (3) CometColumnarToRow InputAdapter - CometHashAggregate [h10_to_10_30,count,count(1)] + CometHashAggregate [count] [h10_to_10_30,count(1)] CometExchange #12 CometHashAggregate [count] CometProject @@ -106,7 +106,7 @@ WholeStageCodegen (8) WholeStageCodegen (4) CometColumnarToRow InputAdapter - CometHashAggregate [h10_30_to_11,count,count(1)] + CometHashAggregate [count] [h10_30_to_11,count(1)] CometExchange #15 CometHashAggregate [count] CometProject @@ -129,7 +129,7 @@ WholeStageCodegen (8) WholeStageCodegen (5) CometColumnarToRow InputAdapter - CometHashAggregate [h11_to_11_30,count,count(1)] + CometHashAggregate [count] [h11_to_11_30,count(1)] CometExchange #18 CometHashAggregate [count] CometProject @@ -152,7 +152,7 @@ WholeStageCodegen (8) WholeStageCodegen (6) CometColumnarToRow InputAdapter - CometHashAggregate [h11_30_to_12,count,count(1)] + CometHashAggregate [count] [h11_30_to_12,count(1)] CometExchange #21 CometHashAggregate [count] CometProject @@ -175,7 +175,7 @@ WholeStageCodegen (8) WholeStageCodegen (7) CometColumnarToRow InputAdapter - CometHashAggregate [h12_to_12_30,count,count(1)] + CometHashAggregate [count] [h12_to_12_30,count(1)] CometExchange #24 CometHashAggregate [count] CometProject 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 f108322730..4fc9029de0 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla CometColumnarToRow InputAdapter CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] 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] 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 510cf60bd8..558f5f4b36 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 @@ -5,9 +5,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #1 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -18,9 +18,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #2 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -31,9 +31,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #3 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -44,9 +44,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #4 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -57,9 +57,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #5 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] 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 2db3b6c957..0991e4e0dc 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 @@ -3,7 +3,7 @@ WholeStageCodegen (2) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [amc,count,count(1)] + CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -32,7 +32,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [pmc,count,count(1)] + CometHashAggregate [count] [pmc,count(1)] CometExchange #6 CometHashAggregate [count] CometProject 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 5a9cc8b447..99c7e48bfb 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) CometColumnarToRow InputAdapter CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] 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] 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 f3ba83cd74..93926eeca5 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometExchange #1 - CometHashAggregate [sum,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [sum] CometProject [ws_ext_discount_amt] CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] CometProject [ws_ext_discount_amt,ws_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count,avg(UnscaledValue(ws_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] 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] 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 d761953c97..d1de4f3475 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] CometExchange [ss_customer_sk] #1 - CometHashAggregate [ss_customer_sk,sum,isEmpty,act_sales] + CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] 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] 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 068a535b6b..bf85b844c9 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 6f07040b87..3e9cdb81f5 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 36f4009885..bbbd07245e 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 49b28b94be..a7de047b25 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,9 +1,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,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 - CometHashAggregate [sum,sum,sum,customer_sk,customer_sk] + CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] CometProject [customer_sk,customer_sk] CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] CometSort [customer_sk,item_sk] 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 3d9267e933..3c91801f00 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 @@ -18,7 +18,7 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 2b9df4aa5a..22896ee566 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] 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] 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 0a2d7a0a29..c745a5b255 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_store_sk] CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] @@ -30,12 +30,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_store_sk,ctr_store_sk,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return),ctr_store_sk] CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_store_sk,sum,count,ctr_total_return] - CometHashAggregate [ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum,sum(UnscaledValue(sr_return_amt))] + CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] + CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_customer_sk,sr_store_sk,sum,sr_return_amt] + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] 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] 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 a3bacdf562..4599ecf539 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -33,9 +33,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -58,9 +58,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -74,9 +74,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 0a1a7d8020..fb83fd2f9a 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 a1a0200121..8ef882a435 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,9 +1,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 - CometHashAggregate [sum,count,sum,count,sum,count,sum,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] 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] 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 13c79b4299..79c782f2ca 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] @@ -13,9 +13,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #15 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 ReusedExchange [d_date_sk] #11 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -116,9 +116,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -133,9 +133,9 @@ WholeStageCodegen (1) CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 5d5557bc59..fb9abae378 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -119,9 +119,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 6eac0d3bd7..c39b96efe3 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [ca_zip,sum(cs_sales_price),sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] CometExchange [ca_zip] #1 - CometHashAggregate [ca_zip,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [ca_zip,sum] 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] 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 352d9e22b1..c703a75e78 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] 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] 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 3a978f4afb..1d48f96b09 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [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(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))] CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] 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] 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 bab3eee57c..6c2b8b2e4f 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)))] + CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,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)))] CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometExpand [i_item_id,ca_country,ca_state,ca_county] [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country,i_item_sk,i_item_id] 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 94d8c234cb..93ab89c142 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] 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] 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 420fd43d85..852c5fca0f 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [d_week_seq] #2 - CometHashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum,d_day_name,sales_price] + CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] 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] @@ -28,7 +28,7 @@ WholeStageCodegen (1) CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 CometBroadcastExchange [d_week_seq] #6 CometProject [d_week_seq] 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 fdf7f7ffa6..2958d060fe 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #1 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 7583431e56..1c2e80c991 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [w_warehouse_name,i_item_id,inv_before,inv_after,sum,sum,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] + CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,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)] CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [w_warehouse_name,i_item_id,sum,sum,d_date,inv_quantity_on_hand] + CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] 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] 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 acf94de869..9119ee749d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] 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] 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 cd74cced17..8f1bddf6c0 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(sales),sum,isEmpty,sum(sales)] + CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] CometExchange #1 - CometHashAggregate [sum,isEmpty,sales] + CometHashAggregate [sales] [sum,isEmpty] CometUnion [sales] CometProject [cs_quantity,cs_list_price] [sales] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] @@ -25,7 +25,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -56,12 +56,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -81,9 +81,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -111,6 +111,6 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt index 0575c4a672..92563114ae 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [c_last_name,c_first_name,sales] CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] @@ -28,7 +28,7 @@ WholeStageCodegen (1) CometBroadcastExchange [item_sk] #4 CometProject [item_sk] CometFilter [item_sk,cnt] - CometHashAggregate [item_sk,cnt,_groupingexpression,i_item_sk,d_date,count,count(1)] + CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,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] @@ -59,12 +59,12 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [tpcds_cmax,max,max(csales)] + CometHashAggregate [max] [tpcds_cmax,max(csales)] CometExchange #11 - CometHashAggregate [max,csales] - CometHashAggregate [csales,c_customer_sk,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [csales] [max] + CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #12 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -84,9 +84,9 @@ WholeStageCodegen (1) CometProject [d_date_sk] CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] CometExchange [c_customer_sk] #9 - CometHashAggregate [c_customer_sk,sum,isEmpty,ss_quantity,ss_sales_price] + CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] 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] @@ -106,15 +106,15 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 CometBroadcastExchange [d_date_sk] #17 CometProject [d_date_sk] CometFilter [d_date_sk,d_year,d_moy] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [c_last_name,c_first_name,sales,sum,isEmpty,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] + CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [c_last_name,c_first_name,sum,isEmpty,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] @@ -132,7 +132,7 @@ WholeStageCodegen (1) CometProject [c_customer_sk] CometFilter [c_customer_sk,ssales] ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [c_customer_sk,ssales,sum,isEmpty,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] + CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] ReusedExchange [c_customer_sk,sum,isEmpty] #9 ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt index c2693d0fb5..dcad304452 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_net_profit,sr_net_loss,cs_net_profit] + CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 dd6b1c575b..0b56a47547 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] + CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 47dd8a4f50..bdae0cc477 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] 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] 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 e8fa7836e1..15b992d3a4 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum,ss_quantity,sr_return_quantity,cs_quantity] + CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] 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] 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 f28011f206..7bdad4b94f 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 016567ca45..da5ca97c85 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_total_return] CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] @@ -36,12 +36,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,ctr_state,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] CometExchange [ctr_state] #6 - CometHashAggregate [ctr_state,sum,count,ctr_total_return] - CometHashAggregate [ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(wr_return_amt))] + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_returning_customer_sk,ca_state,sum,wr_return_amt] + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] 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] 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 4f065b40bd..83d717e295 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] 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 435da991de..2f97384320 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] @@ -11,9 +11,9 @@ WholeStageCodegen (1) CometProject [ca_county,d_year,store_sales,store_sales,store_sales] CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [ca_county,d_year,store_sales,d_qoy,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -34,9 +34,9 @@ WholeStageCodegen (1) CometFilter [ca_address_sk,ca_county] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -55,9 +55,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [ca_county,store_sales,d_qoy,d_year,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -76,9 +76,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -89,9 +89,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year,d_qoy] #4 ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] @@ -102,9 +102,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year,d_qoy] #9 ReusedExchange [ca_address_sk,ca_county] #5 CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [ca_county,web_sales,d_qoy,d_year,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ca_county,d_qoy,d_year,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] 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] 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 a89a381578..07619fc999 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [excess discount amount,sum,sum(UnscaledValue(cs_ext_discount_amt))] + CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 - CometHashAggregate [sum,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [sum] CometProject [cs_ext_discount_amt] CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] CometProject [cs_ext_discount_amt,cs_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count,avg(UnscaledValue(cs_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_item_sk,sum,count,cs_ext_discount_amt] + CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] 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] 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 cd8521b4eb..ff7856615f 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [i_manufact_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] CometExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] CometUnion [i_manufact_id,total_sales] - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_manufact_id] #2 - CometHashAggregate [i_manufact_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] 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] @@ -41,9 +41,9 @@ WholeStageCodegen (1) CometProject [i_manufact_id] [i_manufact_id] CometFilter [i_category,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_manufact_id] #8 - CometHashAggregate [i_manufact_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] 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] @@ -63,9 +63,9 @@ WholeStageCodegen (1) CometProject [i_manufact_id] CometFilter [i_category,i_manufact_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [i_manufact_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_manufact_id] #11 - CometHashAggregate [i_manufact_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] 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] 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 ceab9d3f0a..287e323952 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 a7c1215d1c..47d0e35dd2 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 d7926e49ee..61782e3011 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 - CometHashAggregate [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] 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] 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 c9ecc235c5..7ceccf5d2c 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 4a5376e4f7..2cd1f70c8a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 4a5376e4f7..2cd1f70c8a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] @@ -38,9 +38,9 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,w_warehouse_name,n,avg,m2,sum,count,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] + CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] 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] 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 47e4022f45..36be965b48 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 @@ -12,9 +12,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -36,9 +36,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] + CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] + CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -77,9 +77,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] + CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] 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] @@ -94,9 +94,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk,d_year] #9 CometBroadcastExchange [customer_id,year_total] #16 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] @@ -110,9 +110,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] + CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] 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] 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 23945dce83..79e3556750 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,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)] CometExchange [w_state,i_item_id] #1 - CometHashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty,d_date,cs_sales_price,cr_refunded_cash] + CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] 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] 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 bff847188c..99c935aaa4 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 @@ -13,7 +13,7 @@ WholeStageCodegen (1) CometBroadcastExchange [i_manufact] #2 CometProject [i_manufact] CometFilter [item_cnt,i_manufact] - CometHashAggregate [item_cnt,i_manufact,count,count(1)] + CometHashAggregate [count] [item_cnt,i_manufact,count(1)] CometExchange [i_manufact] #3 CometHashAggregate [i_manufact,count] CometProject [i_manufact] [i_manufact] 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 9de0f79dee..839937d40c 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [d_year,i_category_id,i_category,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] 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] 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 c7445ea60c..59ad7611a5 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] 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] 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 70452e4c3a..e3ec416397 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 @@ -23,15 +23,15 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing] WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [rank_col,ss_store_sk,sum,count,avg(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_store_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] CometProject [ss_store_sk,ss_net_profit] CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_item_sk,sum,count,ss_net_profit] + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] CometProject [ss_item_sk,ss_net_profit] CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] 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 ab17b13248..e646cb620a 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ss_addr_sk,ca_city,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] 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] 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 71df9be678..19124a9e20 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 b456a3d4a3..e29e101c7e 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [sum(ss_quantity),sum,sum(ss_quantity)] + CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] CometExchange #1 - CometHashAggregate [sum,ss_quantity] + CometHashAggregate [ss_quantity] [sum] CometProject [ss_quantity] CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] CometProject [ss_quantity,ss_sold_date_sk] 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 b967ae28f0..e86e0869f5 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [s_store_id] #2 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -48,7 +48,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -72,7 +72,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometColumnarToRow InputAdapter CometExchange [web_site_id] #8 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 a85fe8c06f..98a44eaf40 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 - CometHashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum,sr_returned_date_sk,ss_sold_date_sk] + CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] 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] 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 30c94463ad..a5af2e114b 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 @@ -21,9 +21,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -51,9 +51,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 18bcdb314f..218062c3ef 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [d_year,i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] 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] 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 b8716d034f..0c123eb728 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] InputAdapter CometSort [i_manufact_id,sum_sales,_w0] CometExchange [i_manufact_id] #1 - CometHashAggregate [i_manufact_id,sum_sales,_w0,d_qoy,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [i_manufact_id,d_qoy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] 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] 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 ea21e4f051..ae0cecb1ba 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 @@ -2,12 +2,12 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [segment,num_customers,segment_base,count,count(1)] + CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] CometExchange [segment] #1 CometHashAggregate [segment,count] - CometHashAggregate [segment,c_customer_sk,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [c_customer_sk] #2 - CometHashAggregate [c_customer_sk,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] 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] 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 dde0a8ebc7..dd7658cbac 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [i_brand,i_brand_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] 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] 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 88b9477d1d..19eca782cd 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] CometUnion [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -42,9 +42,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_id] CometFilter [i_item_id,i_color] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #4 ReusedExchange [ca_address_sk] #5 ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 b131b85ea9..b854e818db 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 8524266dc5..2e709c1b54 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometProject [item_id,ss_item_rev,cs_item_rev] CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] CometFilter [item_id,ss_item_rev] - CometHashAggregate [item_id,ss_item_rev,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -48,9 +48,9 @@ WholeStageCodegen (1) ReusedExchange [d_date] #3 CometBroadcastExchange [item_id,cs_item_rev] #6 CometFilter [item_id,cs_item_rev] - CometHashAggregate [item_id,cs_item_rev,i_item_id,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -80,9 +80,9 @@ WholeStageCodegen (1) ReusedExchange [d_date] #9 CometBroadcastExchange [item_id,ws_item_rev] #11 CometFilter [item_id,ws_item_rev] - CometHashAggregate [item_id,ws_item_rev,i_item_id,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #12 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 d6ad4a9650..bca9c6463d 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,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))] CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum,d_day_name,ss_sales_price] + CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] 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] @@ -32,7 +32,7 @@ WholeStageCodegen (1) CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 CometBroadcastExchange [s_store_sk,s_store_id] #6 CometProject [s_store_id] [s_store_sk,s_store_id] 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 35d3166cef..2c772b5998 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -53,9 +53,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_current_price,i_category] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 914f53e34b..086da3b776 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,sum,isEmpty,sum(total_sales)] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,isEmpty,total_sales] + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] CometUnion [i_item_id,total_sales] - CometHashAggregate [i_item_id,total_sales,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #2 - CometHashAggregate [i_item_id,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] 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] @@ -42,9 +42,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_id] CometFilter [i_item_id,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #4 ReusedExchange [ca_address_sk] #5 ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,total_sales,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id] #9 - CometHashAggregate [i_item_id,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] 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] 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 d5d8f9b8f0..6ebe25b753 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 @@ -3,9 +3,9 @@ WholeStageCodegen (2) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [promotions,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] @@ -56,9 +56,9 @@ WholeStageCodegen (2) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [total,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 - CometHashAggregate [sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] CometProject [ss_item_sk,ss_ext_sales_price] 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 a530489ce4..dee2e434c7 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum,ws_ship_date_sk,ws_sold_date_sk] + CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] 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] 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 1b89499528..02166879a9 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] InputAdapter CometSort [i_manager_id,sum_sales,_w0] CometExchange [i_manager_id] #1 - CometHashAggregate [i_manager_id,sum_sales,_w0,d_moy,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [i_manager_id,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] 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] 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 ed591f262b..cbe790cd00 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] 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 6f1dea6cef..43527978d8 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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 + CometExchange [product_name,store_name,cnt] #1 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] CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] @@ -117,8 +117,8 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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] CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -171,7 +171,7 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 CometBroadcastExchange [d_date_sk,d_year] #23 CometFilter [d_date_sk,d_year] 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 2e8a46fe81..cd1c84b6f2 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 @@ -12,9 +12,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [ss_store_sk,ss_item_sk,revenue,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] @@ -37,12 +37,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] CometBroadcastExchange [ss_store_sk,ave] #6 CometFilter [ss_store_sk,ave] - CometHashAggregate [ss_store_sk,ave,sum,count,avg(revenue)] + CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] CometExchange [ss_store_sk] #7 - CometHashAggregate [ss_store_sk,sum,count,revenue] - CometHashAggregate [ss_store_sk,revenue,ss_item_sk,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [revenue] [ss_store_sk,sum,count] + CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_store_sk,ss_item_sk,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] 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] 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 07fd441ba4..bc6bde24b2 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [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(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] + CometHashAggregate [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] [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] CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] + CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] 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] @@ -41,9 +41,9 @@ WholeStageCodegen (1) CometProject [sm_ship_mode_sk] CometFilter [sm_ship_mode_sk,sm_carrier] CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END)] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,d_year,sum(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)] CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] + CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] 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] 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 4ac6889711..e85c243077 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ 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 - 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))] + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] 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] 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 46feb4b4e9..fdddd82de0 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ss_addr_sk,ca_city,sum,sum,sum,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] + CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] 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] 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 395b135acb..a15bcd5c0d 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] 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 ea88d5805a..d6ba61a844 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 @@ -56,9 +56,9 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometSort [s_state,_w0] - CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #7 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 a083a4398d..63e7c353ac 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] CometExchange [ext_price,brand_id] #1 - CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] + CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] + CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 c8f7f27fba..227dd00f2b 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 @@ -6,7 +6,7 @@ WholeStageCodegen (1) CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 400468d3e1..72e39422e0 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 aa9d0e4801..d0520c8b63 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -82,9 +82,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 eed5e27107..b5449831cb 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum,ext_sales_price] + CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] 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] 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 821894c08d..a1243769e5 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 @@ -14,9 +14,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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))] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,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] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -39,9 +39,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [s_store_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,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] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -59,9 +59,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,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] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -69,9 +69,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] ReusedExchange [d_date_sk] #4 CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -82,9 +82,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,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] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -97,9 +97,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,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] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 fbe810689f..77c05217f4 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 @@ -7,9 +7,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -36,9 +36,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -57,9 +57,9 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholes ReusedExchange [d_date_sk,d_year] #5 CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 b3905b01c2..8e244d6c9b 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 @@ -4,9 +4,9 @@ WholeStageCodegen (1) CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_ticket_number,ss_customer_sk,s_city,amt,profit,ss_addr_sk,sum,sum,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum,ss_coupon_amt,ss_net_profit] + CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] 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] 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 50673ff9c1..d1542ab607 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [s_store_name,sum(ss_net_profit),sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] CometExchange [s_store_name] #1 - CometHashAggregate [s_store_name,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_store_name,sum] 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] @@ -40,7 +40,7 @@ WholeStageCodegen (1) CometBroadcastExchange [ca_zip] #7 CometProject [ca_zip] CometFilter [ca_zip,cnt] - CometHashAggregate [ca_zip,cnt,ca_zip,count,count(1)] + CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] CometExchange [ca_zip] #8 CometHashAggregate [ca_zip,count] CometProject [ca_zip] 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 d403cf479b..2c56b98787 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -63,7 +63,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -96,7 +96,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 9296d5436e..a3a981dc54 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 @@ -9,9 +9,9 @@ WholeStageCodegen (1) CometProject [ctr_customer_sk,ctr_total_return] CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] @@ -36,12 +36,12 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [(avg(ctr_total_return) * 1.2),ctr_state,ctr_state,sum,count,avg(ctr_total_return)] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] CometExchange [ctr_state] #6 - CometHashAggregate [ctr_state,sum,count,ctr_total_return] - CometHashAggregate [ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_returning_customer_sk,ca_state,sum,cr_return_amt_inc_tax] + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] 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] 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 825d31a9d4..bff9c86587 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 @@ -6,9 +6,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] CometProject [item_id,sr_item_qty,cr_item_qty] CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [item_id,sr_item_qty,i_item_id,sum,sum(sr_return_quantity)] + CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] CometExchange [i_item_id] #1 - CometHashAggregate [i_item_id,sum,sr_return_quantity] + CometHashAggregate [sr_return_quantity] [i_item_id,sum] 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] @@ -43,9 +43,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [item_id,cr_item_qty,i_item_id,sum,sum(cr_return_quantity)] + CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] CometExchange [i_item_id] #8 - CometHashAggregate [i_item_id,sum,cr_return_quantity] + CometHashAggregate [cr_return_quantity] [i_item_id,sum] 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] @@ -74,9 +74,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #10 CometBroadcastExchange [item_id,wr_item_qty] #12 - CometHashAggregate [item_id,wr_item_qty,i_item_id,sum,sum(wr_return_quantity)] + CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] CometExchange [i_item_id] #13 - CometHashAggregate [i_item_id,sum,wr_return_quantity] + CometHashAggregate [wr_return_quantity] [i_item_id,sum] 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] 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 52cf7b57a0..dfcca32f03 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] CometExchange [r_reason_desc] #1 - CometHashAggregate [r_reason_desc,sum,count,sum,count,sum,count,ws_quantity,wr_refunded_cash,wr_fee] + CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] 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] 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 92a7eb7e8a..29e2d72920 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] CometExchange [_w1,_w2] #1 - CometHashAggregate [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,spark_grouping_id,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [i_category,i_class,spark_grouping_id,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] 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] 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 3490714189..e3923fb3ea 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 @@ -8,7 +8,7 @@ WholeStageCodegen (8) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [h8_30_to_9,count,count(1)] + CometHashAggregate [count] [h8_30_to_9,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -37,7 +37,7 @@ WholeStageCodegen (8) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [h9_to_9_30,count,count(1)] + CometHashAggregate [count] [h9_to_9_30,count(1)] CometExchange #6 CometHashAggregate [count] CometProject @@ -60,7 +60,7 @@ WholeStageCodegen (8) WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [h9_30_to_10,count,count(1)] + CometHashAggregate [count] [h9_30_to_10,count(1)] CometExchange #9 CometHashAggregate [count] CometProject @@ -83,7 +83,7 @@ WholeStageCodegen (8) WholeStageCodegen (3) CometColumnarToRow InputAdapter - CometHashAggregate [h10_to_10_30,count,count(1)] + CometHashAggregate [count] [h10_to_10_30,count(1)] CometExchange #12 CometHashAggregate [count] CometProject @@ -106,7 +106,7 @@ WholeStageCodegen (8) WholeStageCodegen (4) CometColumnarToRow InputAdapter - CometHashAggregate [h10_30_to_11,count,count(1)] + CometHashAggregate [count] [h10_30_to_11,count(1)] CometExchange #15 CometHashAggregate [count] CometProject @@ -129,7 +129,7 @@ WholeStageCodegen (8) WholeStageCodegen (5) CometColumnarToRow InputAdapter - CometHashAggregate [h11_to_11_30,count,count(1)] + CometHashAggregate [count] [h11_to_11_30,count(1)] CometExchange #18 CometHashAggregate [count] CometProject @@ -152,7 +152,7 @@ WholeStageCodegen (8) WholeStageCodegen (6) CometColumnarToRow InputAdapter - CometHashAggregate [h11_30_to_12,count,count(1)] + CometHashAggregate [count] [h11_30_to_12,count(1)] CometExchange #21 CometHashAggregate [count] CometProject @@ -175,7 +175,7 @@ WholeStageCodegen (8) WholeStageCodegen (7) CometColumnarToRow InputAdapter - CometHashAggregate [h12_to_12_30,count,count(1)] + CometHashAggregate [count] [h12_to_12_30,count(1)] CometExchange #24 CometHashAggregate [count] CometProject 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 140fe297a1..61bfd1d960 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_cla 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 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] 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] 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 510cf60bd8..558f5f4b36 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 @@ -5,9 +5,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #1 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -18,9 +18,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #2 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -31,9 +31,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #3 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -44,9 +44,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #4 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] @@ -57,9 +57,9 @@ WholeStageCodegen (1) 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))] + CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] CometExchange #5 - CometHashAggregate [count,sum,count,sum,count,ss_ext_discount_amt,ss_net_paid] + CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] CometProject [ss_ext_discount_amt,ss_net_paid] CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] 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 2db3b6c957..0991e4e0dc 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 @@ -3,7 +3,7 @@ WholeStageCodegen (2) BroadcastNestedLoopJoin CometColumnarToRow InputAdapter - CometHashAggregate [amc,count,count(1)] + CometHashAggregate [count] [amc,count(1)] CometExchange #1 CometHashAggregate [count] CometProject @@ -32,7 +32,7 @@ WholeStageCodegen (2) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [pmc,count,count(1)] + CometHashAggregate [count] [pmc,count(1)] CometExchange #6 CometHashAggregate [count] CometProject 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 a1b8084d84..3e9b8945da 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] CometExchange [Returns_Loss] #1 - CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] + CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] 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] 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 7a1825cc9b..4ec5755df7 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,9 +1,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [Excess Discount Amount ,sum,sum(UnscaledValue(ws_ext_discount_amt))] + CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 - CometHashAggregate [sum,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [sum] CometProject [ws_ext_discount_amt] CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] CometProject [ws_ext_discount_amt,ws_sold_date_sk] @@ -26,9 +26,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count,avg(UnscaledValue(ws_ext_discount_amt))] + CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_item_sk,sum,count,ws_ext_discount_amt] + CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] 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] 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 d761953c97..d1de4f3475 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [ss_customer_sk,sumsales,sum,isEmpty,sum(act_sales)] + CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] CometExchange [ss_customer_sk] #1 - CometHashAggregate [ss_customer_sk,sum,isEmpty,act_sales] + CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] 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] 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 068a535b6b..bf85b844c9 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 6f07040b87..3e9cdb81f5 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 @@ -8,7 +8,7 @@ WholeStageCodegen (2) 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] + CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] 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] 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 36f4009885..bbbd07245e 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,7 +1,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [count(1),count,count(1)] + CometHashAggregate [count] [count(1),count(1)] CometExchange #1 CometHashAggregate [count] CometProject 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 49b28b94be..a7de047b25 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,9 +1,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,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 - CometHashAggregate [sum,sum,sum,customer_sk,customer_sk] + CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] CometProject [customer_sk,customer_sk] CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] CometSort [customer_sk,item_sk] 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 f563efa55d..af05a7d8c6 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 @@ -13,9 +13,9 @@ WholeStageCodegen (3) InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] CometExchange [i_class] #2 - CometHashAggregate [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 2b9df4aa5a..22896ee566 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 @@ -2,9 +2,9 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [sum,sum,sum,sum,sum] [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(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)] CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum,cs_ship_date_sk,cs_sold_date_sk] + CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] 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] 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 08c0ad906c..38f41d10a5 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [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(1)] CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 9d22d7d913..157d1d587e 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 ad8631ea1d..58ac81ba0d 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 5d5557bc59..fb9abae378 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -119,9 +119,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 047cf5ecfe..758e917335 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 @@ -6,18 +6,18 @@ WholeStageCodegen (1) CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -44,9 +44,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #18 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -127,9 +127,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -143,9 +143,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -157,23 +157,23 @@ WholeStageCodegen (1) ReusedExchange [ss_item_sk] #5 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 ReusedExchange [d_date_sk] #15 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id] #22 - CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,sum_sales,number_sales,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] + CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel] #23 - CometHashAggregate [channel,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,sum_sales,number_sales,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange #24 - CometHashAggregate [sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [sum_sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt index 7f696df1df..1572a2a240 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [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] @@ -51,9 +51,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -78,9 +78,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -105,9 +105,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id] #13 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -132,9 +132,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange #15 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] 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 51d992e9cb..cf18e68a3d 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 18f195566a..b59605103e 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 @@ -3,11 +3,11 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] 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] @@ -35,23 +35,23 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [i_product_name,i_brand,i_class,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,qoh,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] + CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [i_product_name,i_brand,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,qoh,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] + CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name] #8 - CometHashAggregate [i_product_name,sum,count,qoh] - CometHashAggregate [i_product_name,qoh,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,sum,count] + CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange #9 - CometHashAggregate [sum,count,qoh] - CometHashAggregate [qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt index 32558f7ba8..0d18ca5626 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id,s_state] #1 - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,s_state,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] @@ -40,9 +40,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometFilter [s_store_sk,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange #9 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] 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] 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 692b84d605..50af5f4f75 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 9ffdb57abc..d57afd4fcf 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 0aa670f8e8..69e8d4868c 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [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(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)] CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 bde96d3e10..e91b278c14 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 @@ -8,13 +8,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter 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] + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] @@ -43,13 +43,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [s_store_sk] CometFilter [s_store_sk,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] CometExchange [i_category] #8 - CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] - CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] CometExchange #9 - CometHashAggregate [sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] - CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt index cc42e6a095..a62c33ecc1 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 172baee830..b3013059b0 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 @@ -34,9 +34,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -83,9 +83,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 7df61c1995..d655789fe6 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 21f646d144..f97451c0c9 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -56,7 +56,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -80,7 +80,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [web_site_id] #9 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 9e3765b84c..cbaf71ab0d 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -53,9 +53,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_current_price,i_category] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 1990c685ce..077daeca3a 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] 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 1dc9f0e3a6..46fe063430 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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 + CometExchange [product_name,store_name,cnt,s1,s1] #1 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] CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] @@ -117,8 +117,8 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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] CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -171,7 +171,7 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 CometBroadcastExchange [d_date_sk,d_year] #23 CometFilter [d_date_sk,d_year] 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 cbcff0282c..6096ee1c25 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 @@ -15,9 +15,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ 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,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))] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] 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] @@ -46,43 +46,43 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan [native_iceberg_compat] 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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt index 614e2c3014..2832c5a348 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter 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] + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,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] @@ -62,9 +62,9 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometSort [s_state,_w0] - CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #8 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 400468d3e1..72e39422e0 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 aa9d0e4801..d0520c8b63 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -82,9 +82,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 ddcfbd3832..ac3d312ee8 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 @@ -22,9 +22,9 @@ WholeStageCodegen (22) 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))] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,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] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -47,9 +47,9 @@ WholeStageCodegen (22) CometFilter [s_store_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,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] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -67,9 +67,9 @@ WholeStageCodegen (22) WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,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] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -77,9 +77,9 @@ WholeStageCodegen (22) ReusedExchange [d_date_sk] #5 CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #11 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -90,9 +90,9 @@ WholeStageCodegen (22) 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,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] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -105,9 +105,9 @@ WholeStageCodegen (22) CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,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] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 ffef4b8107..273db28e77 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 @@ -7,9 +7,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -36,9 +36,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -57,9 +57,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ReusedExchange [d_date_sk,d_year] #5 CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 967ed43de1..f3a5ab15b6 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -71,7 +71,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -104,7 +104,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 9c46912086..11e3c03d80 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 @@ -8,13 +8,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter 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] + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] CometProject [ws_net_paid,i_class,i_category] CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] @@ -37,13 +37,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [i_class,i_category] [i_item_sk,i_class,i_category] CometFilter [i_item_sk,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] CometExchange [i_category] #7 - CometHashAggregate [i_category,sum,isEmpty,total_sum] - CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] CometExchange #8 - CometHashAggregate [sum,isEmpty,total_sum] - CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt index 196e3744a7..46191f59cd 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 @@ -12,9 +12,9 @@ WholeStageCodegen (3) InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 08c0ad906c..38f41d10a5 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [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(1)] CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 e763cabd7b..397a5f6da9 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -38,7 +38,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -67,7 +67,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -87,7 +87,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 27e2d3e64a..95364332b9 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 1c69144d59..6b5264b76d 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -121,9 +121,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 047cf5ecfe..758e917335 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 @@ -6,18 +6,18 @@ WholeStageCodegen (1) CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -44,9 +44,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #18 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -127,9 +127,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -143,9 +143,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -157,23 +157,23 @@ WholeStageCodegen (1) ReusedExchange [ss_item_sk] #5 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 ReusedExchange [d_date_sk] #15 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id] #22 - CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,sum_sales,number_sales,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] + CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel] #23 - CometHashAggregate [channel,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,sum_sales,number_sales,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange #24 - CometHashAggregate [sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [sum_sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt index 7f696df1df..1572a2a240 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [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] @@ -51,9 +51,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -78,9 +78,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -105,9 +105,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id] #13 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -132,9 +132,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange #15 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] 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 344ef0da5d..1b6a5f13f3 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 fb906acb33..d68adf3747 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 @@ -3,11 +3,11 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(qoh)] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(qoh)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] 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] @@ -35,23 +35,23 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [i_product_name,i_brand,i_class,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,qoh,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] + CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [i_product_name,i_brand,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,qoh,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] + CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name] #8 - CometHashAggregate [i_product_name,sum,count,qoh] - CometHashAggregate [i_product_name,qoh,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,sum,count] + CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange #9 - CometHashAggregate [sum,count,qoh] - CometHashAggregate [qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt index 32558f7ba8..0d18ca5626 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id,s_state] #1 - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,s_state,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] @@ -40,9 +40,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometFilter [s_store_sk,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange #9 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] 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] 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 a526a8835c..94be39468a 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 9ffdb57abc..d57afd4fcf 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 0aa670f8e8..69e8d4868c 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [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(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)] CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 c32b26674b..b799504154 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter 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] + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 WholeStageCodegen (6) HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] @@ -19,7 +19,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometColumnarToRow InputAdapter CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] 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 890277a36b..b08d56b5ca 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, CometColumnarToRow InputAdapter CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 63c7e00860..c88883dbad 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 @@ -39,7 +39,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometColumnarToRow InputAdapter CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -91,7 +91,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store CometColumnarToRow InputAdapter CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 12a7a78eba..29a73f88ac 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m CometColumnarToRow InputAdapter CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] 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 21f646d144..f97451c0c9 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -56,7 +56,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -80,7 +80,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [web_site_id] #9 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 98b6bac0cb..010720bc0e 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt] CometFilter [state,cnt] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -55,9 +55,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_category,i_current_price] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] 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 c236c65822..e2784d3e5b 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (185) +- CometSort (184) - +- CometColumnarExchange (183) + +- CometExchange (183) +- CometProject (182) +- CometSortMergeJoin (181) :- CometSort (112) @@ -1013,9 +1013,9 @@ Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#1 Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -(183) CometColumnarExchange +(183) CometExchange Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (184) CometSort Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] 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 e3dda89163..29748bfd47 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 @@ -2,7 +2,7 @@ WholeStageCodegen (37) 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 + CometExchange [product_name,store_name,cnt,s1,s1] #1 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] @@ -72,7 +72,7 @@ WholeStageCodegen (37) CometColumnarToRow InputAdapter CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] 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 cbcff0282c..6096ee1c25 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 @@ -15,9 +15,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ 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,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))] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] 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] @@ -46,43 +46,43 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan [native_iceberg_compat] 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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,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-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index af6f5c49d3..21c6dd4eb1 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter 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] + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,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] @@ -64,7 +64,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometExchange [s_state] #8 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 28d1e0cad6..6db4583da6 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 @@ -10,7 +10,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -38,7 +38,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -67,7 +67,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -87,7 +87,7 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] CometColumnarToRow InputAdapter CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 94b40c8419..9a3700b0b1 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 @@ -12,7 +12,7 @@ WholeStageCodegen (3) CometColumnarToRow InputAdapter CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -90,7 +90,7 @@ WholeStageCodegen (3) CometColumnarToRow InputAdapter CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 f94b690d47..fba0dcde79 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 @@ -24,7 +24,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange [s_store_sk] #3 - CometHashAggregate [s_store_sk,sum,sum,ss_ext_sales_price,ss_net_profit] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -53,7 +53,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange [s_store_sk] #8 - CometHashAggregate [s_store_sk,sum,sum,sr_return_amt,sr_net_loss] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -73,7 +73,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_call_center_sk,sum,sum,cs_ext_sales_price,cs_net_profit] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -83,7 +83,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange #11 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -96,7 +96,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange [wp_web_page_sk] #12 - CometHashAggregate [wp_web_page_sk,sum,sum,ws_ext_sales_price,ws_net_profit] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -115,7 +115,7 @@ WholeStageCodegen (28) CometColumnarToRow InputAdapter CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wp_web_page_sk,sum,sum,wr_return_amt,wr_net_loss] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 3bde32fe52..850bab3c02 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometColumnarToRow InputAdapter CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -46,7 +46,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometColumnarToRow InputAdapter CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -71,7 +71,7 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometColumnarToRow InputAdapter CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 967ed43de1..f3a5ab15b6 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -71,7 +71,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -104,7 +104,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 c8e5b8dd67..d9db895fba 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter 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] + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 WholeStageCodegen (6) HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] @@ -19,7 +19,7 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometColumnarToRow InputAdapter CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] CometProject [ws_net_paid,i_class,i_category] CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] 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 4cbecb6a66..1a79190465 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 @@ -17,7 +17,7 @@ WholeStageCodegen (4) CometColumnarToRow InputAdapter CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 08c0ad906c..38f41d10a5 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [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(1)] CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] CometProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 9d22d7d913..157d1d587e 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] + CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum,ss_ext_list_price,ss_ext_discount_amt] + CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] + CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,ws_ext_list_price,ws_ext_discount_amt] + CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] 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] 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 ad8631ea1d..58ac81ba0d 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ws_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ws_ext_sales_price] + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 5d5557bc59..fb9abae378 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #14 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -27,9 +27,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -119,9 +119,9 @@ WholeStageCodegen (1) CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #4 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] 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 047cf5ecfe..758e917335 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 @@ -6,18 +6,18 @@ WholeStageCodegen (1) CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sales,number_sales] + CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] Subquery #3 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [average_sales,sum,count,avg((cast(quantity as decimal(10,0)) * list_price))] + CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] CometExchange #16 - CometHashAggregate [sum,count,quantity,list_price] + CometHashAggregate [quantity,list_price] [sum,count] 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] @@ -44,9 +44,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #4 ReusedExchange [d_date_sk] #18 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ss_quantity,ss_list_price] + CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -127,9 +127,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,cs_quantity,cs_list_price] + CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -143,9 +143,9 @@ WholeStageCodegen (1) ReusedExchange [d_date_sk] #15 CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] ReusedSubquery [average_sales] #3 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum,isEmpty,count,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] + CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count,ws_quantity,ws_list_price] + CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] 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] @@ -157,23 +157,23 @@ WholeStageCodegen (1) ReusedExchange [ss_item_sk] #5 ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 ReusedExchange [d_date_sk] #15 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] + CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel,i_brand_id] #22 - CometHashAggregate [channel,i_brand_id,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,sum_sales,number_sales,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] + CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange [channel] #23 - CometHashAggregate [channel,sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [channel,sum_sales,number_sales,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] + CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum,sum(sum_sales),sum(number_salesL)] + CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] CometExchange #24 - CometHashAggregate [sum,isEmpty,sum,sum_sales,number_sales] - CometHashAggregate [sum_sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum,sum(sales),sum(number_salesL)] + CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] + CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt index 7f696df1df..1572a2a240 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [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] @@ -51,9 +51,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -78,9 +78,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -105,9 +105,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange [i_item_id] #13 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] @@ -132,9 +132,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] ReusedExchange [d_date_sk] #7 ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] CometExchange #15 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] 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] 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 51d992e9cb..cf18e68a3d 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 @@ -8,9 +8,9 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #1 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(cs_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,cs_ext_sales_price] + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] 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 18f195566a..b59605103e 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 @@ -3,11 +3,11 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count,inv_quantity_on_hand] + CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] 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] @@ -35,23 +35,23 @@ WholeStageCodegen (1) CometBroadcastExchange [w_warehouse_sk] #5 CometFilter [w_warehouse_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [i_product_name,i_brand,i_class,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,i_class,qoh,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] + CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [i_product_name,i_brand,sum,count,qoh] - CometHashAggregate [i_product_name,i_brand,qoh,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] + CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange [i_product_name] #8 - CometHashAggregate [i_product_name,sum,count,qoh] - CometHashAggregate [i_product_name,qoh,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [i_product_name,sum,count] + CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [i_product_name,i_brand,i_class,i_category,qoh,sum,count,avg(qoh)] + CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] CometExchange #9 - CometHashAggregate [sum,count,qoh] - CometHashAggregate [qoh,i_product_name,i_brand,i_class,i_category,sum,count,avg(inv_quantity_on_hand)] + CometHashAggregate [qoh] [sum,count] + CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt index 32558f7ba8..0d18ca5626 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 @@ -3,9 +3,9 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id,s_state] #1 - CometHashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] CometProject [i_item_id,s_state,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] @@ -40,9 +40,9 @@ WholeStageCodegen (1) CometProject [i_item_id] [i_item_sk,i_item_id] CometFilter [i_item_sk,i_item_id] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange [i_item_id] #7 - CometHashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] 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] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometFilter [s_store_sk,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] + CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] CometExchange #9 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,agg1,agg2,agg3,agg4] + CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] 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] 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 692b84d605..50af5f4f75 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] 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 9ffdb57abc..d57afd4fcf 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 @@ -2,11 +2,11 @@ WholeStageCodegen (1) 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 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [ss_ticket_number,ss_customer_sk,cnt,count,count(1)] + CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] CometExchange [ss_ticket_number,ss_customer_sk] #2 CometHashAggregate [ss_ticket_number,ss_customer_sk,count] CometProject [ss_customer_sk,ss_ticket_number] 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 0aa670f8e8..69e8d4868c 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [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(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)] CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 CometHashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] CometProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 bde96d3e10..e91b278c14 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 @@ -8,13 +8,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i InputAdapter 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] + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,sum,ss_net_profit,ss_ext_sales_price] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] @@ -43,13 +43,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i CometProject [s_store_sk] CometFilter [s_store_sk,s_state] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] CometExchange [i_category] #8 - CometHashAggregate [i_category,sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] - CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] CometExchange #9 - CometHashAggregate [sum,isEmpty,sum,isEmpty,ss_net_profit,ss_ext_sales_price] - CometHashAggregate [ss_net_profit,ss_ext_sales_price,i_category,i_class,sum,sum,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index cc42e6a095..a62c33ecc1 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, 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 - CometHashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 43ce5f13af..ca80833ee5 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 @@ -22,9 +22,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,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] + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -62,9 +62,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,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] + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] @@ -92,9 +92,9 @@ WholeStageCodegen (11) 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))] + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,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] + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] 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] 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 172baee830..b3013059b0 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 @@ -34,9 +34,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_item_sk,d_date,sum,ws_sales_price] + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] 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] @@ -83,9 +83,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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))] + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_item_sk,d_date,sum,ss_sales_price] + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] 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] 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 7df61c1995..d655789fe6 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 @@ -17,9 +17,9 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum,cs_sales_price] + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] 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] @@ -56,7 +56,7 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum,sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 InputAdapter BroadcastExchange #9 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 21f646d144..f97451c0c9 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [s_store_id] #3 - CometHashAggregate [s_store_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] 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] @@ -56,7 +56,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [cp_catalog_page_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] 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] @@ -80,7 +80,7 @@ WholeStageCodegen (19) CometColumnarToRow InputAdapter CometExchange [web_site_id] #9 - CometHashAggregate [web_site_id,sum,sum,sum,sum,sales_price,return_amt,profit,net_loss] + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] 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] 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 9e3765b84c..cbaf71ab0d 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 @@ -3,7 +3,7 @@ WholeStageCodegen (1) InputAdapter CometTakeOrderedAndProject [state,cnt,ca_state] CometFilter [state,cnt,ca_state] - CometHashAggregate [state,cnt,ca_state,count,count(1)] + CometHashAggregate [count] [state,cnt,ca_state,count(1)] CometExchange [ca_state] #1 CometHashAggregate [ca_state,count] CometProject [ca_state] @@ -53,9 +53,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] CometBroadcastExchange [avg(i_current_price),i_category] #8 CometFilter [avg(i_current_price),i_category] - CometHashAggregate [avg(i_current_price),i_category,sum,count,avg(UnscaledValue(i_current_price))] + CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] CometExchange [i_category] #9 - CometHashAggregate [i_category,sum,count,i_current_price] + CometHashAggregate [i_current_price] [i_category,sum,count] CometProject [i_category] [i_current_price,i_category] CometFilter [i_current_price,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 1990c685ce..077daeca3a 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,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] 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 1dc9f0e3a6..46fe063430 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 @@ -2,13 +2,13 @@ WholeStageCodegen (1) 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 + CometExchange [product_name,store_name,cnt,s1,s1] #1 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] CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -61,9 +61,9 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_item_sk,sum,sum,isEmpty,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] 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] @@ -117,8 +117,8 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] 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] CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum,ss_wholesale_cost,ss_list_price,ss_coupon_amt] + CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] + CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [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] CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] @@ -171,7 +171,7 @@ WholeStageCodegen (1) CometSort [cs_item_sk] CometProject [cs_item_sk] CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [cs_item_sk,sale,refund,sum,sum,isEmpty,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] + CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 CometBroadcastExchange [d_date_sk,d_year] #23 CometFilter [d_date_sk,d_year] 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 f62a24fa6d..5ca258aeca 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 @@ -9,9 +9,9 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] CometExchange [i_category] #1 CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,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] + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] 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] @@ -40,43 +40,43 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] CometScan [native_iceberg_compat] 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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,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)] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,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))] + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 72d0bd53d8..8de8b7e637 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 @@ -8,7 +8,7 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count InputAdapter 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] + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,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] @@ -61,9 +61,9 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count CometColumnarToRow InputAdapter CometSort [s_state,_w0] - CometHashAggregate [s_state,_w0,sum,sum(UnscaledValue(ss_net_profit))] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #8 - CometHashAggregate [s_state,sum,ss_net_profit] + CometHashAggregate [ss_net_profit] [s_state,sum] 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] 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 c770348a8e..678f8c67df 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) 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)] + CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] CometProject [w_warehouse_name,i_item_desc,d_week_seq] 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 400468d3e1..72e39422e0 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 @@ -8,9 +8,9 @@ WholeStageCodegen (1) CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -32,9 +32,9 @@ WholeStageCodegen (1) CometFilter [d_date_sk,d_year] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ss_net_paid))] + CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ss_net_paid] + CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -57,9 +57,9 @@ WholeStageCodegen (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometBroadcastExchange [customer_id,year_total] #10 CometFilter [customer_id,year_total] - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] @@ -73,9 +73,9 @@ WholeStageCodegen (1) ReusedSubquery [d_date_sk] #1 ReusedExchange [d_date_sk,d_year] #4 CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [customer_id,year_total,c_customer_id,c_first_name,c_last_name,d_year,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] 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] 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 aa9d0e4801..d0520c8b63 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 @@ -7,9 +7,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] @@ -82,9 +82,9 @@ WholeStageCodegen (1) CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum,sum,sum(sales_cnt),sum(UnscaledValue(sales_amt))] + CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum,sales_cnt,sales_amt] + CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] 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 CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] 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 ddcfbd3832..ac3d312ee8 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 @@ -22,9 +22,9 @@ WholeStageCodegen (22) 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))] + CometHashAggregate [sum,sum] [s_store_sk,sales,profit,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] + CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] 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] @@ -47,9 +47,9 @@ WholeStageCodegen (22) CometFilter [s_store_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,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] + CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] 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] @@ -67,9 +67,9 @@ WholeStageCodegen (22) WholeStageCodegen (2) CometColumnarToRow InputAdapter - CometHashAggregate [cs_call_center_sk,sales,profit,sum,sum,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] + CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,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] + CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] @@ -77,9 +77,9 @@ WholeStageCodegen (22) ReusedExchange [d_date_sk] #5 CometColumnarToRow InputAdapter - CometHashAggregate [returns,profit_loss,sum,sum,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #11 - CometHashAggregate [sum,sum,cr_return_amount,cr_net_loss] + CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] @@ -90,9 +90,9 @@ WholeStageCodegen (22) 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,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] + CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] 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] @@ -105,9 +105,9 @@ WholeStageCodegen (22) CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] 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))] + CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,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] + CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] 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] 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 ffef4b8107..273db28e77 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 @@ -7,9 +7,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum,sum,sum,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] + CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum,ss_quantity,ss_wholesale_cost,ss_sales_price] + CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] 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] @@ -36,9 +36,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum,sum,sum,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] + CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum,ws_quantity,ws_wholesale_cost,ws_sales_price] + CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] 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] @@ -57,9 +57,9 @@ TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp ReusedExchange [d_date_sk,d_year] #5 CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum,sum,sum,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] + CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum,cs_quantity,cs_wholesale_cost,cs_sales_price] + CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] 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] 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 967ed43de1..f3a5ab15b6 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 @@ -22,7 +22,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -71,7 +71,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] 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] @@ -104,7 +104,7 @@ WholeStageCodegen (19) 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] + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] 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] 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 9c46912086..11e3c03d80 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 @@ -8,13 +8,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl InputAdapter 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] + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] CometExchange [i_category,i_class] #3 - CometHashAggregate [i_category,i_class,sum,ws_net_paid] + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] CometProject [ws_net_paid,i_class,i_category] CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] CometProject [ws_item_sk,ws_net_paid] @@ -37,13 +37,13 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_cl CometProject [i_class,i_category] [i_item_sk,i_class,i_category] CometFilter [i_item_sk,i_class,i_category] CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] CometExchange [i_category] #7 - CometHashAggregate [i_category,sum,isEmpty,total_sum] - CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty,sum(total_sum)] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] CometExchange #8 - CometHashAggregate [sum,isEmpty,total_sum] - CometHashAggregate [total_sum,i_category,i_class,sum,sum(UnscaledValue(ws_net_paid))] + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 196e3744a7..46191f59cd 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 @@ -12,9 +12,9 @@ WholeStageCodegen (3) InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] CometExchange [i_class] #2 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum,sum(UnscaledValue(ss_ext_sales_price))] + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum,ss_ext_sales_price] + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] 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] From 1a1d3292c07a807a57ff30fcd09153832d26cad0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 11:31:36 -0400 Subject: [PATCH 24/35] new plans. --- docs/source/user-guide/latest/configs.md | 2 +- .../approved-plans-v1_4-spark3_5/q71/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q91/simplified.txt | 2 +- .../tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt | 2 +- .../tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index fd2a1c8c55..25ddba31ef 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -88,6 +88,6 @@ Comet provides the following configuration settings. | spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | | spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. Note that this config is only used when `spark.comet.exec.shuffle.mode` is `jvm`. | 10.0 | | spark.comet.shuffle.sizeInBytesMultiplier | Comet reports smaller sizes for shuffle due to using Arrow's columnar memory format and this can result in Spark choosing a different join strategy due to the estimated size of the exchange being smaller. Comet will multiple sizeInBytes by this amount to avoid regressions in join strategy. | 1.0 | -| spark.comet.sparkToColumnar.supportedOperatorList | A comma-separated list of operators that will be converted to Arrow columnar format when 'spark.comet.sparkToColumnar.enabled' is true | Range,InMemoryTableScan | +| spark.comet.sparkToColumnar.supportedOperatorList | A comma-separated list of operators that will be converted to Arrow columnar format when 'spark.comet.sparkToColumnar.enabled' is true | Range,InMemoryTableScan,RDDScan | | spark.hadoop.fs.comet.libhdfs.schemes | Defines filesystem schemes (e.g., hdfs, webhdfs) that the native side accesses via libhdfs, separated by commas. Valid only when built with hdfs feature enabled. | | 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 1dbf04331b..63e7c353ac 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] 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 575d92032a..3e9b8945da 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] 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 1dbf04331b..63e7c353ac 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] 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 575d92032a..3e9b8945da 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 @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] From 6456d4e6b869fc9d90cfbf6ae6d20e5823f85955 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 16:17:37 -0400 Subject: [PATCH 25/35] New tests to replicate Spark SQL failure. --- .../comet/exec/CometNativeShuffleSuite.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index c3d9c2b3da..dc5495a41b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -248,6 +248,27 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } + // This duplicates behavior seen in a much more complicated Spark SQL test + // group-analytics.sql + test("range partitioning with GROUPING functions should not cause ClassCastException") { + // Reproduces the issue where Cast expressions in ORDER BY cause ClassCastException + // when trying to cast to AttributeReference in RangePartitioning deduplication logic + withParquetTable( + Seq(("Math", 2020), ("Math", 2021), ("Physics", 2020), ("Physics", 2021)), + "courseSales") { + val df = sql(""" + SELECT _1, _2, GROUPING(_1), GROUPING(_2) + FROM courseSales + GROUP BY CUBE(_1, _2) + ORDER BY GROUPING(_1), GROUPING(_2), _1, _2 + """) + + // This should not throw ClassCastException during RangePartitioning + // The ORDER BY with GROUPING functions creates Cast expressions that cause the issue + df.repartitionByRange(2, col("_1")).collect() + } + } + // This duplicates behavior seen in a much more complicated Spark SQL test // "SPARK-44647: test join key is the second cluster key" test("range partitioning with duplicate column references") { From e24a36b0903627e25d8917f3d05579b4cd09942b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 16:32:25 -0400 Subject: [PATCH 26/35] switch to using hash-based deduplication like DataFusion's LexOrdering. This fixes an issue where the child was always cast to an AttributeReference which might not be the case for grouping expressions which insert Casts. Added a test case for the issue too. --- .../execution/shuffle/CometNativeShuffleWriter.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 6e6aba703b..5205297431 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -219,19 +219,17 @@ class CometNativeShuffleWriter[K, V]( val partitioning = PartitioningOuterClass.RangePartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) - // Detect duplicates by tracking bound references to same exprId + // Detect duplicates by tracking expressions directly, similar to DataFusion's LexOrdering // DataFusion will deduplicate identical sort expressions in LexOrdering, // so we need to transform boundary rows to match the deduplicated structure - val seenExprIds = mutable.HashSet[Long]() + val seenExprs = mutable.HashSet[org.apache.spark.sql.catalyst.expressions.Expression]() val deduplicationMap = mutable.ArrayBuffer[(Int, Boolean)]() // (originalIndex, isKept) rangePartitioning.ordering.zipWithIndex.foreach { case (sortOrder, idx) => - val attr = sortOrder.child.asInstanceOf[AttributeReference] - - if (seenExprIds.contains(attr.exprId.id)) { + if (seenExprs.contains(sortOrder.child)) { deduplicationMap += (idx -> false) // Will be deduplicated by DataFusion } else { - seenExprIds += attr.exprId.id + seenExprs += sortOrder.child deduplicationMap += (idx -> true) // Will be kept by DataFusion } } From b7a078a7114cf1a4a2029f54ef161458949ff0cc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 16:38:10 -0400 Subject: [PATCH 27/35] Minor refactor. --- .../comet/execution/shuffle/CometNativeShuffleWriter.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 5205297431..fd1207cb23 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriter} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, SinglePartition} import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric @@ -222,7 +222,7 @@ class CometNativeShuffleWriter[K, V]( // Detect duplicates by tracking expressions directly, similar to DataFusion's LexOrdering // DataFusion will deduplicate identical sort expressions in LexOrdering, // so we need to transform boundary rows to match the deduplicated structure - val seenExprs = mutable.HashSet[org.apache.spark.sql.catalyst.expressions.Expression]() + val seenExprs = mutable.HashSet[Expression]() val deduplicationMap = mutable.ArrayBuffer[(Int, Boolean)]() // (originalIndex, isKept) rangePartitioning.ordering.zipWithIndex.foreach { case (sortOrder, idx) => From 4233b42a471bad4c5ae010188d966f256833f00d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 20 Sep 2025 20:18:42 -0400 Subject: [PATCH 28/35] add benchmark. --- .../sql/benchmark/CometShuffleBenchmark.scala | 134 ++++++++++++++++++ 1 file changed, 134 insertions(+) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala index 46af7115cc..103866331c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala @@ -421,6 +421,76 @@ object CometShuffleBenchmark extends CometBenchmarkBase { } } + def shuffleRangePartitionBenchmark( + values: Int, + dataType: DataType, + width: Int, + partitionNum: Int): Unit = { + val benchmark = + new Benchmark( + s"SQL Wide ($width cols) ${dataType.sql} Range Partition Shuffle($partitionNum Partition)", + values, + output = output) + + val projection = (1 to width) + .map(i => s"CAST(CAST(RAND(1) * 100 AS INTEGER) AS ${dataType.sql}) AS c$i") + .mkString(", ") + val columns = (1 to width).map(i => s"c$i").mkString(", ") + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT $projection FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + + benchmark.addCase("SQL Parquet - Comet (Spark Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "false") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Comet Arrow Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Comet Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "native") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.run() + } + } + } + override def runCometBenchmark(mainArgs: Array[String]): Unit = { runBenchmarkWithTable("Shuffle on array", 1024 * 1024 * 1) { v => Seq( @@ -591,5 +661,69 @@ object CometShuffleBenchmark extends CometBenchmarkBase { shuffleWideBenchmark(v, dataType, 20, 201) } } + + runBenchmarkWithTable("Wide Range Partition Shuffle (10 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 10, 5) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (20 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 20, 5) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (10 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 10, 201) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (20 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 20, 201) + } + } } } From b5b286b50af584dc99373a183962f4b27fb7de0c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 22 Sep 2025 11:04:20 -0400 Subject: [PATCH 29/35] Remove development test, add more tests for duplicates in columns and random data. --- .../comet/exec/CometNativeShuffleSuite.scala | 130 ++++++++++-------- 1 file changed, 72 insertions(+), 58 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index dc5495a41b..a682ff91a5 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -20,13 +20,14 @@ package org.apache.comet.exec import scala.concurrent.duration.DurationInt +import scala.util.Random import org.scalactic.source.Position import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv -import org.apache.spark.sql.{CometTestBase, DataFrame, Row} +import org.apache.spark.sql.{CometTestBase, DataFrame, Dataset, Row} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col @@ -248,27 +249,6 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } - // This duplicates behavior seen in a much more complicated Spark SQL test - // group-analytics.sql - test("range partitioning with GROUPING functions should not cause ClassCastException") { - // Reproduces the issue where Cast expressions in ORDER BY cause ClassCastException - // when trying to cast to AttributeReference in RangePartitioning deduplication logic - withParquetTable( - Seq(("Math", 2020), ("Math", 2021), ("Physics", 2020), ("Physics", 2021)), - "courseSales") { - val df = sql(""" - SELECT _1, _2, GROUPING(_1), GROUPING(_2) - FROM courseSales - GROUP BY CUBE(_1, _2) - ORDER BY GROUPING(_1), GROUPING(_2), _1, _2 - """) - - // This should not throw ClassCastException during RangePartitioning - // The ORDER BY with GROUPING functions creates Cast expressions that cause the issue - df.repartitionByRange(2, col("_1")).collect() - } - } - // This duplicates behavior seen in a much more complicated Spark SQL test // "SPARK-44647: test join key is the second cluster key" test("range partitioning with duplicate column references") { @@ -307,6 +287,45 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } + // Asserts ordering properties of partitions in a Dataset that has been RangePartitioned + private def checkRangePartitionedDataset(df_range_partitioned: Dataset[Row]): Unit = { + val partition_bounds = df_range_partitioned.rdd + .mapPartitionsWithIndex((idx: Int, iterator: Iterator[Row]) => { + // Find the min and max value in each partition + var min: Option[Int] = None + var max: Option[Int] = None + iterator.foreach((row: Row) => { + val row_val = row.get(0).asInstanceOf[Int] + if (min.isEmpty || row_val < min.get) { + min = Some(row_val) + } + if (max.isEmpty || row_val > max.get) { + max = Some(row_val) + } + }) + Iterator.single((idx, min, max)) + }) + .collect() + + // Check min and max values in each partition + for (i <- partition_bounds.indices.init) { + val currentPartition = partition_bounds(i) + val nextPartition = partition_bounds(i + 1) + + if (currentPartition._2.isDefined && currentPartition._3.isDefined) { + val currentMin = currentPartition._2.get + val currentMax = currentPartition._3.get + assert(currentMin <= currentMax) + } + + if (currentPartition._3.isDefined && nextPartition._2.isDefined) { + val currentMax = currentPartition._3.get + val nextMin = nextPartition._2.get + assert(currentMax < nextMin) + } + } + } + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to // test for incorrect partition values after native RangePartitioning test("fix: range partitioning #1906") { @@ -315,47 +334,42 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper val df = sql("SELECT * from tbl") // Repartition with two sort columns - val df_range_partitioned = df.repartitionByRange(10, df.col("_1"), df.col("_2")) - - val partition_bounds = df_range_partitioned.rdd - .mapPartitionsWithIndex((idx: Int, iterator: Iterator[Row]) => { - // Find the min and max value in each partition - var min: Option[Int] = None - var max: Option[Int] = None - iterator.foreach((row: Row) => { - val row_val = row.get(0).asInstanceOf[Int] - if (min.isEmpty || row_val < min.get) { - min = Some(row_val) - } - if (max.isEmpty || row_val > max.get) { - max = Some(row_val) - } - }) - Iterator.single((idx, min, max)) - }) - .collect() - - // Check min and max values in each partition - for (i <- partition_bounds.indices.init) { - val currentPartition = partition_bounds(i) - val nextPartition = partition_bounds(i + 1) - - if (currentPartition._2.isDefined && currentPartition._3.isDefined) { - val currentMin = currentPartition._2.get - val currentMax = currentPartition._3.get - assert(currentMin <= currentMax) - } + val repartitioned_df = df.repartitionByRange(10, df.col("_1")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) + } + } + } - if (currentPartition._3.isDefined && nextPartition._2.isDefined) { - val currentMax = currentPartition._3.get - val nextMin = nextPartition._2.get - assert(currentMax <= nextMin) - } - } + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906, two columns") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + withParquetTable((0 until 100000).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT * from tbl") + // Repartition with two sort columns + val repartitioned_df = df.repartitionByRange(10, df.col("_1"), df.col("_2")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) } } + } + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906, random sort column with duplicates") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + val random = new Random(42) + withParquetTable((0 until 100000).map(i => (random.nextInt(10000), i)), "tbl") { + val df = sql("SELECT * from tbl") + + // Repartition with two sort columns + val repartitioned_df = df.repartitionByRange(10, df.col("_1")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) + } + } } /** From fa2c20b920ec36a18a3b53a1eebb39f80ebc251c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 23 Sep 2025 15:43:47 -0400 Subject: [PATCH 30/35] Update docs. --- common/src/main/scala/org/apache/comet/CometConf.scala | 5 +---- docs/source/user-guide/latest/configs.md | 2 +- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index ef538f2341..15c160ed6c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -325,12 +325,9 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) - // RangePartitioning contains bugs https://github.com/apache/datafusion-comet/issues/1906 val COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.native.shuffle.partitioning.range.enabled") - .doc("Experimental feature to enable range partitioning for Comet native shuffle. " + - "This feature is experimental while we investigate scenarios that don't partition data " + - "correctly.") + .doc("Whether to enable range partitioning for Comet native shuffle.") .booleanConf .createWithDefault(true) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 25ddba31ef..410386e958 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -72,7 +72,7 @@ Comet provides the following configuration settings. | spark.comet.memoryOverhead | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. This config is optional. If this is not specified, it will be set to `spark.comet.memory.overhead.factor` * `spark.executor.memory`. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | | | spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 | | spark.comet.native.shuffle.partitioning.hash.enabled | Whether to enable hash partitioning for Comet native shuffle. | true | -| spark.comet.native.shuffle.partitioning.range.enabled | Experimental feature to enable range partitioning for Comet native shuffle. This feature is experimental while we investigate scenarios that don't partition data correctly. | true | +| spark.comet.native.shuffle.partitioning.range.enabled | Whether to enable range partitioning for Comet native shuffle. | true | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | | spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false | | spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false | From 8781264332c99df207d6deb7fe0a6d46101848a3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 23 Sep 2025 15:50:01 -0400 Subject: [PATCH 31/35] PR feedback. --- native/core/src/execution/planner.rs | 40 +++++++++++++--------------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 8d0a54fec2..7e704a7e94 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2363,31 +2363,29 @@ impl PhysicalPlanner { let sort_fields: Vec = lex_ordering .iter() .map(|sort_expr| { - let data_type = sort_expr.expr.data_type(input_schema.as_ref()).unwrap(); - SortField::new_with_options(data_type, sort_expr.options) + sort_expr + .expr + .data_type(input_schema.as_ref()) + .map(|dt| SortField::new_with_options(dt, sort_expr.options)) }) - .collect(); + .collect::, _>>()?; // Deserialize the literals to columnar collections of ScalarValues let mut scalar_values: Vec> = vec![vec![]; lex_ordering.len()]; - range_partition - .boundary_rows - .iter() - .for_each(|boundary_row| { - // For each serialized expr in a boundary row, convert to a Literal - // expression, then extract the ScalarValue from the Literal and push it - // into the collection of ScalarValues - (0..lex_ordering.len()).for_each(|col_idx| { - let expr = self - .create_expr( - &boundary_row.partition_bounds[col_idx], - Arc::clone(&input_schema), - ) - .unwrap(); - let literal_expr = expr.as_any().downcast_ref::().unwrap(); - scalar_values[col_idx].push(literal_expr.value().clone()); - }); - }); + for boundary_row in &range_partition.boundary_rows { + // For each serialized expr in a boundary row, convert to a Literal + // expression, then extract the ScalarValue from the Literal and push it + // into the collection of ScalarValues + for col_idx in 0..lex_ordering.len() { + let expr = self.create_expr( + &boundary_row.partition_bounds[col_idx], + Arc::clone(&input_schema), + )?; + let literal_expr = + expr.as_any().downcast_ref::().expect("Literal"); + scalar_values[col_idx].push(literal_expr.value().clone()); + } + } // Convert the collection of ScalarValues to collection of Arrow Arrays let arrays: Vec = scalar_values From 318adbdec65523f42b2bf55db16f95632999cdca Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 23 Sep 2025 18:20:03 -0400 Subject: [PATCH 32/35] Fix clippy. --- native/core/src/execution/planner.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 7e704a7e94..5a18fadf01 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2376,14 +2376,18 @@ impl PhysicalPlanner { // For each serialized expr in a boundary row, convert to a Literal // expression, then extract the ScalarValue from the Literal and push it // into the collection of ScalarValues - for col_idx in 0..lex_ordering.len() { + for (col_idx, col_values) in scalar_values + .iter_mut() + .enumerate() + .take(lex_ordering.len()) + { let expr = self.create_expr( &boundary_row.partition_bounds[col_idx], Arc::clone(&input_schema), )?; let literal_expr = expr.as_any().downcast_ref::().expect("Literal"); - scalar_values[col_idx].push(literal_expr.value().clone()); + col_values.push(literal_expr.value().clone()); } } From 2aa3f0fbc34dcf4796fa2714de899f5d29ad90bd Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 24 Sep 2025 08:28:07 -0400 Subject: [PATCH 33/35] Update comments based on PR feedback. --- native/core/src/execution/planner.rs | 9 ++++++--- native/core/src/execution/shuffle/comet_partitioning.rs | 7 +++++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5a18fadf01..64efa31d52 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2399,14 +2399,17 @@ impl PhysicalPlanner { // Create a RowConverter and use to create OwnedRows from the Arrays let converter = RowConverter::new(sort_fields)?; - let rows = converter.convert_columns(&arrays)?; - let owned_rows: Vec = rows.iter().map(|row| row.owned()).collect(); + let boundary_rows = converter.convert_columns(&arrays)?; + // Rows are only a view into Arrow Arrays. We need to create OwnedRows with their + // own internal memory ownership to pass as our boundary values to the partitioner. + let boundary_owned_rows: Vec = + boundary_rows.iter().map(|row| row.owned()).collect(); Ok(CometPartitioning::RangePartitioning( lex_ordering, range_partition.num_partitions as usize, Arc::new(converter), - owned_rows, + boundary_owned_rows, )) } PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::SinglePartition), diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index 90f9ef9327..a2422cf9e6 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -23,10 +23,13 @@ use std::sync::Arc; pub enum CometPartitioning { SinglePartition, /// Allocate rows based on a hash of one of more expressions and the specified number of - /// partitions + /// partitions. Args are 1) the expression to hash on, and 2) the number of partitions. Hash(Vec>, usize), /// Allocate rows based on the lexical order of one of more expressions and the specified number of - /// partitions + /// partitions. Args are 1) the LexOrdering to use to compare values and split into partitions, + /// 2) the number of partitions, 3) the RowConverter used to view incoming RecordBatches as Arrow + /// Rows for comparing to 4) OwnedRows that represent the boundaries of each partition, used with + /// LexOrdering to bin each value in the RecordBatch to a partition. RangePartitioning(LexOrdering, usize, Arc, Vec), } From 3d359ba0c4e8db266a8b410fcf71930d8eb51a62 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 24 Sep 2025 12:51:38 -0400 Subject: [PATCH 34/35] Update spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala Co-authored-by: Oleks V --- .../comet/execution/shuffle/CometNativeShuffleWriter.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index fd1207cb23..a76b598bb8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -148,12 +148,7 @@ class CometNativeShuffleWriter[K, V]( // Spark sometimes generates RangePartitioning schemes with numPartitions == 1, // or the computed bounds results in a single target partition. // In this case Comet just serializes a SinglePartition scheme to native. - if ((rp.numPartitions == 1) || rangePartitionBounds.isEmpty || - rangePartitionBounds.get.isEmpty) { - true - } else { - false - } + rp.numPartitions == 1 || rangePartitionBounds.forall(_.isEmpty) case hp: HashPartitioning => hp.numPartitions == 1 case _ => false } From c87aba7694cf540febdaf9cf6c732e044f83a77f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 24 Sep 2025 14:11:51 -0400 Subject: [PATCH 35/35] Fix formatting after accepting change on GitHub. --- .../sql/comet/execution/shuffle/CometNativeShuffleWriter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index a76b598bb8..3dfd1f8d03 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -148,7 +148,7 @@ class CometNativeShuffleWriter[K, V]( // Spark sometimes generates RangePartitioning schemes with numPartitions == 1, // or the computed bounds results in a single target partition. // In this case Comet just serializes a SinglePartition scheme to native. - rp.numPartitions == 1 || rangePartitionBounds.forall(_.isEmpty) + rp.numPartitions == 1 || rangePartitionBounds.forall(_.isEmpty) case hp: HashPartitioning => hp.numPartitions == 1 case _ => false }